You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by sr...@apache.org on 2014/06/03 21:44:04 UTC

[2/2] git commit: SENTRY-153: Add Hive e2e test with grant/revoke statements (Prasad Mujumdar via Sravya Tirukkovalur)

SENTRY-153: Add Hive e2e test with grant/revoke statements (Prasad Mujumdar via Sravya Tirukkovalur)


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

Branch: refs/heads/master
Commit: 600f0f15eac08f63ef76eb1c2e66ee761f560cb0
Parents: ac70d73
Author: Sravya Tirukkovalur <sr...@clouera.com>
Authored: Tue Jun 3 12:42:56 2014 -0700
Committer: Sravya Tirukkovalur <sr...@clouera.com>
Committed: Tue Jun 3 12:42:56 2014 -0700

----------------------------------------------------------------------
 pom.xml                                         |   4 +-
 .../hive/ql/exec/SentryGrantRevokeTask.java     |  14 +-
 .../provider/db/SimpleDBProviderBackend.java    |   2 +-
 .../db/service/persistent/SentryStore.java      |   8 +-
 .../thrift/SentryPolicyServiceClient.java       |   9 +-
 .../sentry/service/thrift/SentryService.java    |   4 +
 .../apache/sentry/provider/file/PolicyFile.java |  11 +-
 .../e2e/dbprovider/PolicyProviderForTest.java   | 154 +++++++++++++++++++
 .../tests/e2e/dbprovider/TestDbCrossDbOps.java  |  36 +++++
 .../tests/e2e/dbprovider/TestDbEndToEnd.java    |   6 +-
 .../TestDbExportImportPrivileges.java           |  33 ++++
 .../TestDbMetadataObjectRetrieval.java          |  33 ++++
 .../dbprovider/TestDbMetadataPermissions.java   |  33 ++++
 .../dbprovider/TestDbMovingToProduction.java    |  32 ++++
 .../dbprovider/TestDbPrivilegeAtTransform.java  |  32 ++++
 .../TestDbPrivilegesAtDatabaseScope.java        |  34 ++++
 .../TestDbPrivilegesAtFunctionScope.java        |  33 ++++
 .../TestDbPrivilegesAtTableScope.java           |  33 ++++
 .../TestDbRuntimeMetadataRetrieval.java         |  33 ++++
 .../tests/e2e/dbprovider/TestDbSandboxOps.java  |  32 ++++
 .../e2e/dbprovider/TestDbUriPermissions.java    |  31 ++++
 .../e2e/dbprovider/TestDbUserManagement.java    |  31 ++++
 .../AbstractTestWithStaticConfiguration.java    | 109 ++++++++++++-
 .../apache/sentry/tests/e2e/hive/Context.java   |   5 +-
 .../sentry/tests/e2e/hive/TestCrossDbOps.java   |  27 ++--
 .../e2e/hive/TestExportImportPrivileges.java    |   6 +-
 .../e2e/hive/TestMetadataObjectRetrieval.java   |   8 +-
 .../tests/e2e/hive/TestMetadataPermissions.java |   7 +-
 .../tests/e2e/hive/TestMovingToProduction.java  |   6 +-
 .../e2e/hive/TestPrivilegesAtDatabaseScope.java |   7 +-
 .../e2e/hive/TestPrivilegesAtFunctionScope.java |   6 +-
 .../e2e/hive/TestPrivilegesAtTableScope.java    |   7 +-
 .../e2e/hive/TestRuntimeMetadataRetrieval.java  |   7 +-
 .../sentry/tests/e2e/hive/TestSandboxOps.java   |   8 +-
 .../tests/e2e/hive/TestUriPermissions.java      |   7 +-
 .../tests/e2e/hive/TestUserManagement.java      |  27 ++--
 .../e2e/hive/hiveserver/AbstractHiveServer.java |  13 +-
 .../e2e/hive/hiveserver/HiveServerFactory.java  |  20 +--
 38 files changed, 806 insertions(+), 102 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 490bb01..72e390a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -446,11 +446,11 @@ limitations under the License.
           <configuration>
             <forkMode>always</forkMode>
             <environmentVariables>
-              <HADOOP_CLIENT_OPTS>-Xmx1000m -Dhive.log.dir=./target/</HADOOP_CLIENT_OPTS>
+              <HADOOP_CLIENT_OPTS>-Xmx1500m -Dhive.log.dir=./target/</HADOOP_CLIENT_OPTS>
             </environmentVariables>
             <forkedProcessTimeoutInSeconds>900</forkedProcessTimeoutInSeconds>
             <redirectTestOutputToFile>true</redirectTestOutputToFile>
-            <argLine>-Xms256m -Xmx1g</argLine>
+            <argLine>-Xms512m -Xmx2g</argLine>
           </configuration>
         </plugin>
         <plugin>

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
index df9b0db..4a50bd0 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hive.ql.plan.RevokeDesc;
 import org.apache.hadoop.hive.ql.plan.RoleDDLDesc;
 import org.apache.hadoop.hive.ql.plan.ShowGrantDesc;
 import org.apache.hadoop.hive.ql.plan.api.StageType;
+import org.apache.hadoop.hive.ql.security.authorization.Privilege.PrivilegeType;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.session.SessionState.LogHelper;
 import org.apache.sentry.SentryUserException;
@@ -64,6 +65,7 @@ import org.apache.sentry.core.model.db.AccessURI;
 import org.apache.sentry.core.model.db.Database;
 import org.apache.sentry.core.model.db.Server;
 import org.apache.sentry.core.model.db.Table;
+import org.apache.sentry.core.model.db.AccessConstants;
 import org.apache.sentry.provider.db.service.thrift.SentryPolicyServiceClient;
 import org.apache.sentry.provider.db.service.thrift.TSentryPrivilege;
 import org.apache.sentry.provider.db.service.thrift.TSentryRole;
@@ -526,14 +528,14 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
               sentryClient.grantDatabasePrivilege(subject, princ.getName(), server, dbName);
             } else {
               sentryClient.grantTablePrivilege(subject, princ.getName(), server, dbName,
-                  tableName, privDesc.getPrivilege().getPriv().name());
+                  tableName, toSentryAction(privDesc.getPrivilege().getPriv()));
             }
           } else {
             if (tableName == null) {
               sentryClient.revokeDatabasePrivilege(subject, princ.getName(), server, dbName);
             } else {
               sentryClient.revokeTablePrivilege(subject, princ.getName(), server, dbName,
-                  tableName, privDesc.getPrivilege().getPriv().name());
+                  tableName, toSentryAction(privDesc.getPrivilege().getPriv()));
             }
           }
         }
@@ -556,6 +558,14 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
     return (SentryHivePrivilegeObjectDesc) privSubjectObjDesc;
   }
 
+  private static String toSentryAction(PrivilegeType privilegeType) {
+    if (PrivilegeType.ALL.equals(privilegeType)) {
+      return AccessConstants.ALL;
+    } else {
+      return privilegeType.name();
+    }
+  }
+
   private static DatabaseTable parseDBTable(String obj) throws HiveException {
     String[] dbTab = Iterables.toArray(DB_TBL_SPLITTER.split(obj), String.class);
     if (dbTab.length == 2) {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SimpleDBProviderBackend.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SimpleDBProviderBackend.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SimpleDBProviderBackend.java
index b66037a..326b91d 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SimpleDBProviderBackend.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SimpleDBProviderBackend.java
@@ -78,7 +78,7 @@ public class SimpleDBProviderBackend implements ProviderBackend {
       throw new IllegalStateException("Backend has not been properly initialized");
     }
     try {
-      return ImmutableSet.copyOf(policyServiceClient.listPrivilegesForProvider(groups, roleSet, authorizableHierarchy));
+      return ImmutableSet.copyOf(policyServiceClient.listPrivilegesForProvider(groups, roleSet, null));
     } catch (SentryUserException e) {
       String msg = "Unable to obtain privileges from server: " + e.getMessage();
       LOGGER.error(msg, e);

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryStore.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryStore.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryStore.java
index a7734df..ff656aa 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryStore.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryStore.java
@@ -41,7 +41,6 @@ import javax.jdo.Transaction;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.sentry.core.model.db.AccessConstants;
-import org.apache.sentry.core.model.db.DBModelAuthorizable;
 import org.apache.sentry.core.model.db.DBModelAuthorizable.AuthorizableType;
 import org.apache.sentry.provider.common.ProviderConstants;
 import org.apache.sentry.provider.db.SentryAccessDeniedException;
@@ -64,7 +63,6 @@ import org.datanucleus.store.rdbms.exceptions.MissingTableException;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
-import com.google.common.base.Splitter;
 import com.google.common.base.Strings;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Lists;
@@ -617,7 +615,9 @@ public class SentryStore {
         filters.append("&& serverName == \"" + authHierarchy.getServer().toLowerCase() + "\"");
         if (authHierarchy.getDb() != null) {
           filters.append(" && ((dbName == \"" + authHierarchy.getDb().toLowerCase() + "\") || (dbName == null)) && (URI == null)");
-          if (authHierarchy.getTable() != null) {
+          if ((authHierarchy.getTable() != null)
+              && !AccessConstants.ALL
+                  .equalsIgnoreCase(authHierarchy.getTable())) {
             filters.append(" && ((tableName == \"" + authHierarchy.getTable().toLowerCase() + "\") || (tableName == null)) && (URI == null)");
           }
         }
@@ -625,7 +625,7 @@ public class SentryStore {
           filters.append(" && ((\"" + authHierarchy.getUri() + "\".startsWith(URI)) || (URI == null)) && (dbName == null)");
         }
       }
-
+      System.out.println("Filter String: " + filters.toString());
       query.setFilter(filters.toString());
       List<MSentryPrivilege> privileges = (List<MSentryPrivilege>) query.execute();
       rollbackTransaction = false;

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClient.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClient.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClient.java
index 3660607..ebcec1b 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClient.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClient.java
@@ -103,7 +103,7 @@ public class SentryPolicyServiceClient {
     LOGGER.info("Successfully created client");
   }
 
-  public void createRole(String requestorUserName, String roleName)
+  public synchronized void createRole(String requestorUserName, String roleName)
   throws SentryUserException {
     TCreateSentryRoleRequest request = new TCreateSentryRoleRequest();
     request.setProtocol_version(ThriftConstants.TSENTRY_SERVICE_VERSION_CURRENT);
@@ -156,7 +156,8 @@ public class SentryPolicyServiceClient {
    * @return Set of thrift sentry role objects
    * @throws SentryUserException
    */
-  public Set<TSentryRole> listRolesByGroupName(String requestorUserName,
+  public synchronized Set<TSentryRole> listRolesByGroupName(
+      String requestorUserName,
       String groupName)
   throws SentryUserException {
     TListSentryRolesRequest request = new TListSentryRolesRequest();
@@ -365,7 +366,7 @@ public class SentryPolicyServiceClient {
     }
   }
 
-  public void grantRoleToGroup(String requestorUserName,
+  public synchronized void grantRoleToGroup(String requestorUserName,
       String groupName, String roleName)
   throws SentryUserException {
     TAlterSentryRoleAddGroupsRequest request = new TAlterSentryRoleAddGroupsRequest(ThriftConstants.
@@ -379,7 +380,7 @@ TSENTRY_SERVICE_VERSION_CURRENT, requestorUserName,
     }
   }
 
-  public void revokeRoleFromGroup(String requestorUserName,
+  public synchronized void revokeRoleFromGroup(String requestorUserName,
       String groupName, String roleName)
   throws SentryUserException {
     TAlterSentryRoleDeleteGroupsRequest request = new TAlterSentryRoleDeleteGroupsRequest(ThriftConstants.

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryService.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryService.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryService.java
index a0ac224..eefcb0a 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryService.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryService.java
@@ -315,4 +315,8 @@ public class SentryService implements Runnable {
       });
     }
   }
+
+  public Configuration getConf() {
+    return conf;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/PolicyFile.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/PolicyFile.java b/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/PolicyFile.java
index ad95bb9..ceeb653 100644
--- a/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/PolicyFile.java
+++ b/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/PolicyFile.java
@@ -56,10 +56,13 @@ public class PolicyFile {
 
   private final Map<String, String> databasesToPolicyFiles = Maps.newHashMap();
   private final Multimap<String, String> usersToGroups = ArrayListMultimap.create();
-  private final Multimap<String, String> groupsToRoles = ArrayListMultimap.create();
-  private final Multimap<String, String> rolesToPermissions = ArrayListMultimap.create();
+  protected final Multimap<String, String> groupsToRoles = ArrayListMultimap
+      .create();
+  protected final Multimap<String, String> rolesToPermissions = ArrayListMultimap
+      .create();
 
-  public PolicyFile addRolesToGroup(String groupName, String... roleNames) {
+  public PolicyFile addRolesToGroup(String groupName, String... roleNames)
+      throws Exception {
     return addRolesToGroup(groupName, false, roleNames);
   }
   public PolicyFile addRolesToGroup(String groupName, boolean allowDuplicates, String... roleNames) {
@@ -185,7 +188,7 @@ public class PolicyFile {
   }
 
   //User:Group mapping for the admin user needs to be set separately
-  public static PolicyFile setAdminOnServer1(String admin) {
+  public static PolicyFile setAdminOnServer1(String admin) throws Exception {
     return new PolicyFile()
       .addRolesToGroup(admin, "admin_role")
       .addPermissionsToRole("admin_role", "server=server1");

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/PolicyProviderForTest.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/PolicyProviderForTest.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/PolicyProviderForTest.java
new file mode 100644
index 0000000..79ca387
--- /dev/null
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/PolicyProviderForTest.java
@@ -0,0 +1,154 @@
+/*
+ * 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.tests.e2e.dbprovider;
+
+import static org.apache.sentry.provider.common.ProviderConstants.AUTHORIZABLE_SPLITTER;
+import static org.apache.sentry.provider.common.ProviderConstants.PRIVILEGE_PREFIX;
+import static org.apache.sentry.tests.e2e.hive.StaticUserGroup.ADMIN1;
+import static org.apache.sentry.tests.e2e.hive.StaticUserGroup.ADMINGROUP;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.sentry.SentryUserException;
+import org.apache.sentry.core.model.db.AccessConstants;
+import org.apache.sentry.core.model.db.DBModelAction;
+import org.apache.sentry.core.model.db.DBModelAuthorizable;
+import org.apache.sentry.core.model.db.DBModelAuthorizable.AuthorizableType;
+import org.apache.sentry.policy.db.DBModelAuthorizables;
+import org.apache.sentry.provider.db.service.thrift.SentryPolicyServiceClient;
+import org.apache.sentry.provider.db.service.thrift.TSentryRole;
+import org.apache.sentry.provider.file.PolicyFile;
+import org.apache.tools.ant.util.StringUtils;
+import org.mortbay.log.Log;
+
+import com.google.common.collect.Sets;
+
+public class PolicyProviderForTest extends PolicyFile {
+  protected static final Set<String> ADMIN_GROUP_SET = Sets
+      .newHashSet(ADMINGROUP);
+  private static SentryPolicyServiceClient sentryClient;
+
+  protected SentryPolicyServiceClient getSentryClient() {
+    return sentryClient;
+  }
+
+  protected static void setSentryClient(
+      SentryPolicyServiceClient newSentryClient) {
+    sentryClient = newSentryClient;
+  }
+
+  public static void clearSentryClient() {
+    sentryClient = null;
+  }
+
+  public static PolicyProviderForTest setAdminOnServer1(String admin)
+      throws Exception {
+    PolicyProviderForTest policyFile = new PolicyProviderForTest();
+    policyFile.addRolesToGroup(admin, "admin_role")
+        .addPermissionsToRole("admin_role", "server=server1");
+    return policyFile;
+  }
+
+  public void write(File file) throws Exception {
+    super.write(file);
+    if (!usingSentryService()) {
+      return;
+    }
+
+    // remove existing metadata
+    for (TSentryRole tRole : sentryClient.listRoles(ADMIN1, ADMIN_GROUP_SET)) {
+      sentryClient.dropRole(ADMIN1, tRole.getRoleName());
+    }
+
+    // create roles and add privileges
+    for (Entry<String, Collection<String>> roleEntry : rolesToPermissions
+        .asMap().entrySet()) {
+      sentryClient.createRole(ADMIN1, roleEntry.getKey());
+      for (String privilege : roleEntry.getValue()) {
+        addPrivilege(roleEntry.getKey(), privilege);
+      }
+    }
+
+    // grant roles to groups
+    for (Entry<String, Collection<String>> groupEntry : groupsToRoles.asMap()
+        .entrySet()) {
+      for (String roleNames : groupEntry.getValue()) {
+        for (String roleName : roleNames.split(",")) {
+          try {
+            sentryClient
+                .grantRoleToGroup(ADMIN1, groupEntry.getKey(), roleName);
+          } catch (SentryUserException e) {
+            Log.warn("Error granting role " + roleName + " to group "
+                + groupEntry.getKey());
+          }
+        }
+      }
+    }
+  }
+
+  private void addPrivilege(String roleName, String privileges) throws Exception {
+    String serverName = null, dbName = null, tableName = null, uriPath = null;
+    String action = AccessConstants.ALL;
+
+    for(String section : AUTHORIZABLE_SPLITTER.split(privileges)) {
+      // action is not an authorizeable
+      if(!section.toLowerCase().startsWith(PRIVILEGE_PREFIX)) {
+        DBModelAuthorizable dbAuthorizable = DBModelAuthorizables.from(section);
+        if(dbAuthorizable == null) {
+          throw new IOException("Unknow Auth type " + section);
+        }
+
+        if (AuthorizableType.Server.equals(dbAuthorizable.getAuthzType())) {
+          serverName = dbAuthorizable.getName();
+        } else if (AuthorizableType.Db.equals(dbAuthorizable.getAuthzType())) {
+          dbName = dbAuthorizable.getName();
+        } else if (AuthorizableType.Table.equals(dbAuthorizable.getAuthzType())) {
+          tableName = dbAuthorizable.getName();
+        } else if (AuthorizableType.URI.equals(dbAuthorizable.getAuthzType())) {
+          uriPath = dbAuthorizable.getName();
+        } else {
+          throw new IOException("Unsupported auth type " + dbAuthorizable.getName()
+              + " : " + dbAuthorizable.getTypeName());
+        }
+      } else {
+        action = DBModelAction.valueOf(
+            StringUtils.removePrefix(section, PRIVILEGE_PREFIX).toUpperCase())
+            .toString();
+      }
+    }
+
+    if (tableName != null) {
+      sentryClient.grantTablePrivilege(ADMIN1, roleName, serverName, dbName,
+          tableName, action);
+    } else if (dbName != null) {
+      sentryClient.grantDatabasePrivilege(ADMIN1, roleName, serverName, dbName);
+    } else if (uriPath != null) {
+      sentryClient.grantURIPrivilege(ADMIN1, roleName, serverName, uriPath);
+    } else if (serverName != null) {
+      sentryClient.grantServerPrivilege(ADMIN1, roleName, serverName);
+      ;
+    }
+  }
+
+  private boolean usingSentryService() {
+    return sentryClient != null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbCrossDbOps.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbCrossDbOps.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbCrossDbOps.java
new file mode 100644
index 0000000..6411e2b
--- /dev/null
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbCrossDbOps.java
@@ -0,0 +1,36 @@
+/*
+ * 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.tests.e2e.dbprovider;
+
+import org.apache.sentry.tests.e2e.hive.AbstractTestWithStaticConfiguration;
+import org.apache.sentry.tests.e2e.hive.TestCrossDbOps;
+import org.junit.BeforeClass;
+
+/* Tests privileges at table scope with cross database access */
+
+public class TestDbCrossDbOps extends TestCrossDbOps {
+
+  @BeforeClass
+  public static void setupTestStaticConfiguration() throws Exception{
+    policy_on_hdfs = true;
+    useSentryService = true;
+    AbstractTestWithStaticConfiguration.setupTestStaticConfiguration();
+    PolicyProviderForTest.setSentryClient(AbstractTestWithStaticConfiguration
+        .getSentryClient());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbEndToEnd.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbEndToEnd.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbEndToEnd.java
index ab65cfd..9c0c8b5 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbEndToEnd.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbEndToEnd.java
@@ -26,9 +26,7 @@ import java.sql.Connection;
 import java.sql.ResultSet;
 import java.sql.Statement;
 
-import org.apache.sentry.tests.e2e.hive.Context;
-import org.apache.sentry.tests.e2e.hive.StaticUserGroup;
-import org.junit.After;
+import org.apache.sentry.provider.file.PolicyFile;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -37,6 +35,7 @@ import com.google.common.io.Resources;
 public class TestDbEndToEnd extends AbstractTestWithDbProvider {
   private final String SINGLE_TYPE_DATA_FILE_NAME = "kv1.dat";
   private File dataFile;
+  private PolicyFile policyFile;
 
   @Before
   public void setup() throws Exception {
@@ -45,6 +44,7 @@ public class TestDbEndToEnd extends AbstractTestWithDbProvider {
     FileOutputStream to = new FileOutputStream(dataFile);
     Resources.copy(Resources.getResource(SINGLE_TYPE_DATA_FILE_NAME), to);
     to.close();
+    policyFile = PolicyFile.setAdminOnServer1(ADMINGROUP);
     setupAdmin(context);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbExportImportPrivileges.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbExportImportPrivileges.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbExportImportPrivileges.java
new file mode 100644
index 0000000..78677ba
--- /dev/null
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbExportImportPrivileges.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tests.e2e.dbprovider;
+
+import org.apache.sentry.tests.e2e.hive.AbstractTestWithStaticConfiguration;
+import org.apache.sentry.tests.e2e.hive.TestExportImportPrivileges;
+import org.junit.BeforeClass;
+
+public class TestDbExportImportPrivileges extends TestExportImportPrivileges {
+  @BeforeClass
+  public static void setupTestStaticConfiguration() throws Exception {
+    useSentryService = true;
+    AbstractTestWithStaticConfiguration.setupTestStaticConfiguration();
+    PolicyProviderForTest.setSentryClient(AbstractTestWithStaticConfiguration
+        .getSentryClient());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbMetadataObjectRetrieval.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbMetadataObjectRetrieval.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbMetadataObjectRetrieval.java
new file mode 100644
index 0000000..2701e55
--- /dev/null
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbMetadataObjectRetrieval.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tests.e2e.dbprovider;
+
+import org.apache.sentry.tests.e2e.hive.AbstractTestWithStaticConfiguration;
+import org.apache.sentry.tests.e2e.hive.TestMetadataObjectRetrieval;
+import org.junit.BeforeClass;
+
+public class TestDbMetadataObjectRetrieval extends TestMetadataObjectRetrieval {
+  @BeforeClass
+  public static void setupTestStaticConfiguration() throws Exception {
+    useSentryService = true;
+    AbstractTestWithStaticConfiguration.setupTestStaticConfiguration();
+    PolicyProviderForTest.setSentryClient(AbstractTestWithStaticConfiguration
+        .getSentryClient());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbMetadataPermissions.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbMetadataPermissions.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbMetadataPermissions.java
new file mode 100644
index 0000000..e311e51
--- /dev/null
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbMetadataPermissions.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tests.e2e.dbprovider;
+
+import org.apache.sentry.tests.e2e.hive.AbstractTestWithStaticConfiguration;
+import org.apache.sentry.tests.e2e.hive.TestMetadataPermissions;
+import org.junit.BeforeClass;
+
+public class TestDbMetadataPermissions extends TestMetadataPermissions {
+  @BeforeClass
+  public static void setupTestStaticConfiguration() throws Exception {
+    useSentryService = true;
+    AbstractTestWithStaticConfiguration.setupTestStaticConfiguration();
+    PolicyProviderForTest.setSentryClient(AbstractTestWithStaticConfiguration
+        .getSentryClient());
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbMovingToProduction.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbMovingToProduction.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbMovingToProduction.java
new file mode 100644
index 0000000..4b44e63
--- /dev/null
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbMovingToProduction.java
@@ -0,0 +1,32 @@
+/*
+ * 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.tests.e2e.dbprovider;
+
+import org.apache.sentry.tests.e2e.hive.AbstractTestWithStaticConfiguration;
+import org.apache.sentry.tests.e2e.hive.TestMovingToProduction;
+import org.junit.BeforeClass;
+
+public class TestDbMovingToProduction extends TestMovingToProduction {
+  @BeforeClass
+  public static void setupTestStaticConfiguration() throws Exception {
+    useSentryService = true;
+    AbstractTestWithStaticConfiguration.setupTestStaticConfiguration();
+    PolicyProviderForTest.setSentryClient(AbstractTestWithStaticConfiguration
+        .getSentryClient());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbPrivilegeAtTransform.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbPrivilegeAtTransform.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbPrivilegeAtTransform.java
new file mode 100644
index 0000000..2cbf0a7
--- /dev/null
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbPrivilegeAtTransform.java
@@ -0,0 +1,32 @@
+/*
+ * 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.tests.e2e.dbprovider;
+
+import org.apache.sentry.tests.e2e.hive.AbstractTestWithStaticConfiguration;
+import org.apache.sentry.tests.e2e.hive.TestPrivilegeAtTransform;
+import org.junit.BeforeClass;
+
+public class TestDbPrivilegeAtTransform extends TestPrivilegeAtTransform {
+  @BeforeClass
+  public static void setupTestStaticConfiguration() throws Exception {
+    useSentryService = true;
+    AbstractTestWithStaticConfiguration.setupTestStaticConfiguration();
+    PolicyProviderForTest.setSentryClient(AbstractTestWithStaticConfiguration
+        .getSentryClient());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbPrivilegesAtDatabaseScope.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbPrivilegesAtDatabaseScope.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbPrivilegesAtDatabaseScope.java
new file mode 100644
index 0000000..7ea8166
--- /dev/null
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbPrivilegesAtDatabaseScope.java
@@ -0,0 +1,34 @@
+/*
+ * 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.tests.e2e.dbprovider;
+
+import org.apache.sentry.tests.e2e.hive.AbstractTestWithStaticConfiguration;
+import org.apache.sentry.tests.e2e.hive.TestPrivilegesAtDatabaseScope;
+import org.junit.BeforeClass;
+
+public class TestDbPrivilegesAtDatabaseScope extends
+    TestPrivilegesAtDatabaseScope {
+  @BeforeClass
+  public static void setupTestStaticConfiguration() throws Exception {
+    useSentryService = true;
+    AbstractTestWithStaticConfiguration.setupTestStaticConfiguration();
+    PolicyProviderForTest.setSentryClient(AbstractTestWithStaticConfiguration
+        .getSentryClient());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbPrivilegesAtFunctionScope.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbPrivilegesAtFunctionScope.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbPrivilegesAtFunctionScope.java
new file mode 100644
index 0000000..e12b072
--- /dev/null
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbPrivilegesAtFunctionScope.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tests.e2e.dbprovider;
+
+import org.apache.sentry.tests.e2e.hive.AbstractTestWithStaticConfiguration;
+import org.apache.sentry.tests.e2e.hive.TestPrivilegesAtFunctionScope;
+import org.junit.BeforeClass;
+
+public class TestDbPrivilegesAtFunctionScope extends
+    TestPrivilegesAtFunctionScope {
+  @BeforeClass
+  public static void setupTestStaticConfiguration() throws Exception {
+    useSentryService = true;
+    AbstractTestWithStaticConfiguration.setupTestStaticConfiguration();
+    PolicyProviderForTest.setSentryClient(AbstractTestWithStaticConfiguration
+        .getSentryClient());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbPrivilegesAtTableScope.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbPrivilegesAtTableScope.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbPrivilegesAtTableScope.java
new file mode 100644
index 0000000..db17012
--- /dev/null
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbPrivilegesAtTableScope.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tests.e2e.dbprovider;
+
+import org.apache.sentry.tests.e2e.hive.AbstractTestWithStaticConfiguration;
+import org.apache.sentry.tests.e2e.hive.TestPrivilegesAtTableScope;
+import org.junit.BeforeClass;
+
+public class TestDbPrivilegesAtTableScope extends TestPrivilegesAtTableScope {
+  @BeforeClass
+  public static void setupTestStaticConfiguration() throws Exception {
+    useSentryService = true;
+    AbstractTestWithStaticConfiguration.setupTestStaticConfiguration();
+    PolicyProviderForTest.setSentryClient(AbstractTestWithStaticConfiguration
+        .getSentryClient());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbRuntimeMetadataRetrieval.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbRuntimeMetadataRetrieval.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbRuntimeMetadataRetrieval.java
new file mode 100644
index 0000000..302492a
--- /dev/null
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbRuntimeMetadataRetrieval.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tests.e2e.dbprovider;
+
+import org.apache.sentry.tests.e2e.hive.AbstractTestWithStaticConfiguration;
+import org.apache.sentry.tests.e2e.hive.TestRuntimeMetadataRetrieval;
+import org.junit.BeforeClass;
+
+public class TestDbRuntimeMetadataRetrieval extends
+    TestRuntimeMetadataRetrieval {
+  @BeforeClass
+  public static void setupTestStaticConfiguration() throws Exception {
+    useSentryService = true;
+    AbstractTestWithStaticConfiguration.setupTestStaticConfiguration();
+    PolicyProviderForTest.setSentryClient(AbstractTestWithStaticConfiguration
+        .getSentryClient());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbSandboxOps.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbSandboxOps.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbSandboxOps.java
new file mode 100644
index 0000000..5eef792
--- /dev/null
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbSandboxOps.java
@@ -0,0 +1,32 @@
+/*
+ * 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.tests.e2e.dbprovider;
+
+import org.apache.sentry.tests.e2e.hive.AbstractTestWithStaticConfiguration;
+import org.apache.sentry.tests.e2e.hive.TestSandboxOps;
+import org.junit.BeforeClass;
+
+public class TestDbSandboxOps extends TestSandboxOps {
+  @BeforeClass
+  public static void setupTestStaticConfiguration() throws Exception {
+    useSentryService = true;
+    AbstractTestWithStaticConfiguration.setupTestStaticConfiguration();
+    PolicyProviderForTest.setSentryClient(AbstractTestWithStaticConfiguration
+        .getSentryClient());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbUriPermissions.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbUriPermissions.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbUriPermissions.java
new file mode 100644
index 0000000..f52fb67
--- /dev/null
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbUriPermissions.java
@@ -0,0 +1,31 @@
+/*
+ * 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.tests.e2e.dbprovider;
+
+import org.apache.sentry.tests.e2e.hive.AbstractTestWithStaticConfiguration;
+import org.apache.sentry.tests.e2e.hive.TestUriPermissions;
+import org.junit.BeforeClass;
+
+public class TestDbUriPermissions extends TestUriPermissions {
+  @BeforeClass
+  public static void setupTestStaticConfiguration() throws Exception {
+    useSentryService = true;
+    AbstractTestWithStaticConfiguration.setupTestStaticConfiguration();
+    PolicyProviderForTest.setSentryClient(AbstractTestWithStaticConfiguration
+        .getSentryClient());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbUserManagement.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbUserManagement.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbUserManagement.java
new file mode 100644
index 0000000..e24137d
--- /dev/null
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbUserManagement.java
@@ -0,0 +1,31 @@
+/*
+ * 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.tests.e2e.dbprovider;
+
+import org.apache.sentry.tests.e2e.hive.AbstractTestWithStaticConfiguration;
+import org.junit.BeforeClass;
+
+public class TestDbUserManagement {
+  @BeforeClass
+  public static void setupTestStaticConfiguration() throws Exception {
+    // useSentryService = true;
+    AbstractTestWithStaticConfiguration.setupTestStaticConfiguration();
+    PolicyProviderForTest.setSentryClient(AbstractTestWithStaticConfiguration
+        .getSentryClient());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/AbstractTestWithStaticConfiguration.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/AbstractTestWithStaticConfiguration.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/AbstractTestWithStaticConfiguration.java
index 96e4821..835e547 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/AbstractTestWithStaticConfiguration.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/AbstractTestWithStaticConfiguration.java
@@ -16,12 +16,29 @@
  */
 package org.apache.sentry.tests.e2e.hive;
 
-import com.google.common.collect.Maps;
-import com.google.common.io.Files;
+import java.io.File;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.util.Map;
+import java.util.concurrent.TimeoutException;
+
 import junit.framework.Assert;
+
 import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.sentry.binding.hive.SentryHiveAuthorizationTaskFactoryImpl;
+import org.apache.sentry.provider.db.SimpleDBProviderBackend;
+import org.apache.sentry.provider.db.service.thrift.SentryPolicyServiceClient;
 import org.apache.sentry.provider.file.PolicyFile;
+import org.apache.sentry.service.thrift.SentryService;
+import org.apache.sentry.service.thrift.SentryServiceClientFactory;
+import org.apache.sentry.service.thrift.SentryServiceFactory;
+import org.apache.sentry.service.thrift.ServiceConstants.ClientConfig;
+import org.apache.sentry.service.thrift.ServiceConstants.ServerConfig;
+import org.apache.sentry.tests.e2e.dbprovider.PolicyProviderForTest;
 import org.apache.sentry.tests.e2e.hive.fs.DFS;
 import org.apache.sentry.tests.e2e.hive.fs.DFSFactory;
 import org.apache.sentry.tests.e2e.hive.hiveserver.HiveServer;
@@ -32,11 +49,8 @@ import org.junit.BeforeClass;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.File;
-import java.sql.Connection;
-import java.sql.ResultSet;
-import java.sql.Statement;
-import java.util.Map;
+import com.google.common.collect.Maps;
+import com.google.common.io.Files;
 
 public abstract class AbstractTestWithStaticConfiguration {
   private static final Logger LOGGER = LoggerFactory
@@ -75,7 +89,10 @@ public abstract class AbstractTestWithStaticConfiguration {
       VIEW3 = "view_3",
       INDEX1 = "index_1",
       INDEX2 = "index_2";
+  protected static final String SERVER_HOST = "localhost";
+
   protected static boolean policy_on_hdfs = false;
+  protected static boolean useSentryService = false;
 
   protected static File baseDir;
   protected static File logDir;
@@ -86,11 +103,15 @@ public abstract class AbstractTestWithStaticConfiguration {
   protected static FileSystem fileSystem;
   protected static DFS dfs;
   protected static Map<String, String> properties;
+  protected static SentryService sentryServer;
+  protected static Configuration sentryConf;
+  protected static SentryPolicyServiceClient sentryClient;
   protected Context context;
 
   public Context createContext() throws Exception {
-    return new Context(hiveServer, fileSystem,
+    context = new Context(hiveServer, fileSystem,
         baseDir, confDir, dataDir, policyFileLocation);
+    return context;
   }
   protected void dropDb(String user, String...dbs) throws Exception {
     Connection connection = context.createConnection(user);
@@ -164,6 +185,10 @@ public abstract class AbstractTestWithStaticConfiguration {
       policyURI = policyFileLocation.getPath();
     }
 
+    if (useSentryService) {
+      setupSentryService();
+    }
+
     hiveServer = HiveServerFactory.create(properties, baseDir, confDir, logDir, policyURI, fileSystem);
     hiveServer.start();
   }
@@ -175,6 +200,55 @@ public abstract class AbstractTestWithStaticConfiguration {
     }
   }
 
+  private static void setupSentryService() throws Exception {
+    properties = Maps.newHashMap();
+    sentryConf = new Configuration(false);
+    PolicyFile policyFile = new PolicyFile();
+
+    properties.put(HiveServerFactory.AUTHZ_PROVIDER_BACKEND,
+        SimpleDBProviderBackend.class.getName());
+    properties.put(ConfVars.HIVE_AUTHORIZATION_TASK_FACTORY.varname,
+        SentryHiveAuthorizationTaskFactoryImpl.class.getName());
+    properties
+        .put(ConfVars.HIVE_SERVER2_THRIFT_MIN_WORKER_THREADS.varname, "2");
+    properties.put(ServerConfig.SECURITY_MODE, ServerConfig.SECURITY_MODE_NONE);
+    properties.put(ServerConfig.ADMIN_GROUPS, ADMINGROUP);
+    properties.put(ServerConfig.RPC_ADDRESS, SERVER_HOST);
+    properties.put(ServerConfig.RPC_PORT, String.valueOf(0));
+    properties.put(ServerConfig.SENTRY_VERIFY_SCHEM_VERSION, "false");
+
+    properties.put(ServerConfig.SENTRY_STORE_JDBC_URL,
+        "jdbc:derby:;databaseName=" + baseDir.getPath()
+            + "/sentrystore_db;create=true");
+    properties.put(ServerConfig.SENTRY_STORE_GROUP_MAPPING, ServerConfig.SENTRY_STORE_LOCAL_GROUP_MAPPING);
+    properties.put(ServerConfig.SENTRY_STORE_GROUP_MAPPING_RESOURCE, policyFileLocation.getPath());
+    properties.put(ServerConfig.RPC_MIN_THREADS, "3");
+    for (Map.Entry<String, String> entry : properties.entrySet()) {
+      sentryConf.set(entry.getKey(), entry.getValue());
+    }
+    sentryServer = new SentryServiceFactory().create(sentryConf);
+    properties.put(ClientConfig.SERVER_RPC_ADDRESS, sentryServer.getAddress()
+        .getHostString());
+    sentryConf.set(ClientConfig.SERVER_RPC_ADDRESS, sentryServer.getAddress()
+        .getHostString());
+    properties.put(ClientConfig.SERVER_RPC_PORT,
+        String.valueOf(sentryServer.getAddress().getPort()));
+    sentryConf.set(ClientConfig.SERVER_RPC_PORT,
+        String.valueOf(sentryServer.getAddress().getPort()));
+    startSentryService();
+  }
+
+  private static void startSentryService() throws Exception {
+    sentryServer.start();
+    final long start = System.currentTimeMillis();
+    while (!sentryServer.isRunning()) {
+      Thread.sleep(1000);
+      if (System.currentTimeMillis() - start > 60000L) {
+        throw new TimeoutException("Server did not start after 60 seconds");
+      }
+    }
+  }
+
   @Before
   public void setup() throws Exception{
     dfs.createBaseDir();
@@ -186,6 +260,17 @@ public abstract class AbstractTestWithStaticConfiguration {
       hiveServer.shutdown();
       hiveServer = null;
     }
+
+    if (sentryServer != null) {
+      if (sentryClient != null) {
+        sentryClient.close();
+      }
+      sentryClient = null;
+      sentryServer.stop();
+      sentryServer = null;
+      PolicyProviderForTest.clearSentryClient();
+    }
+
     if(baseDir != null) {
       if(System.getProperty(HiveServerFactory.KEEP_BASEDIR) == null) {
         FileUtils.deleteQuietly(baseDir);
@@ -200,4 +285,12 @@ public abstract class AbstractTestWithStaticConfiguration {
       }
     }
   }
+
+  public static SentryPolicyServiceClient getSentryClient() throws Exception {
+    if (sentryServer == null) {
+      throw new IllegalAccessException("Sentry service not initialized");
+    }
+    SentryServiceClientFactory factory = new SentryServiceClientFactory();
+    return factory.create(sentryServer.getConf());
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/Context.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/Context.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/Context.java
index 4fc0068..39d411e 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/Context.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/Context.java
@@ -65,7 +65,6 @@ public class Context {
   private final Set<Connection> connections;
   private final Set<Statement> statements;
 
-
   public Context(HiveServer hiveServer, FileSystem fileSystem,
       File baseDir, File confDir, File dataDir, File policyFile) throws Exception {
     this.hiveServer = hiveServer;
@@ -138,7 +137,7 @@ public class Context {
     for(Statement statement : statements) {
       try {
         statement.close();
-      } catch (SQLException exception) {
+      } catch (Exception exception) {
         LOGGER.warn("Error closing " + statement, exception);
       }
     }
@@ -147,7 +146,7 @@ public class Context {
     for(Connection connection : connections) {
       try {
         connection.close();
-      } catch (SQLException exception) {
+      } catch (Exception exception) {
         LOGGER.warn("Error closing " + connection, exception);
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestCrossDbOps.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestCrossDbOps.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestCrossDbOps.java
index 64a51e4..84e3b19 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestCrossDbOps.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestCrossDbOps.java
@@ -17,13 +17,8 @@
 
 package org.apache.sentry.tests.e2e.hive;
 
-import com.google.common.io.Resources;
-import junit.framework.Assert;
-import org.apache.sentry.provider.file.PolicyFile;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.io.File;
 import java.io.FileOutputStream;
@@ -35,14 +30,21 @@ import java.sql.Statement;
 import java.util.ArrayList;
 import java.util.List;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import junit.framework.Assert;
+
+import org.apache.sentry.tests.e2e.dbprovider.PolicyProviderForTest;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.io.Resources;
 
 /* Tests privileges at table scope with cross database access */
 
 public class TestCrossDbOps extends AbstractTestWithStaticConfiguration {
   private File dataFile;
-  private PolicyFile policyFile;
+  private PolicyProviderForTest policyFile;
   private String loadData;
 
   @BeforeClass
@@ -60,7 +62,7 @@ public class TestCrossDbOps extends AbstractTestWithStaticConfiguration {
     FileOutputStream to = new FileOutputStream(dataFile);
     Resources.copy(Resources.getResource(SINGLE_TYPE_DATA_FILE_NAME), to);
     to.close();
-    policyFile = PolicyFile.setAdminOnServer1(ADMINGROUP);
+    policyFile = PolicyProviderForTest.setAdminOnServer1(ADMINGROUP);
     loadData = "server=server1->uri=file://" + dataFile.getPath();
 
   }
@@ -375,10 +377,11 @@ public class TestCrossDbOps extends AbstractTestWithStaticConfiguration {
         // Positive case: test user1 and user2 has permissions to access
         // db1 and
         // db2
+        userStmt.execute("Use " + dbName);
         userStmt
         .execute("create table " + dbName + "." + tabName + " (id int)");
         userStmt.execute("LOAD DATA LOCAL INPATH '" + dataFile.getPath()
-            + "' INTO TABLE " + dbName + "." + tabName);
+            + "' INTO TABLE " + tabName);
         userStmt.execute("select * from " + dbName + "." + tabName);
         context.close();
       }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestExportImportPrivileges.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestExportImportPrivileges.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestExportImportPrivileges.java
index 94ec018..411a8e9 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestExportImportPrivileges.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestExportImportPrivileges.java
@@ -24,7 +24,7 @@ import java.sql.Connection;
 import java.sql.Statement;
 
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.sentry.provider.file.PolicyFile;
+import org.apache.sentry.tests.e2e.dbprovider.PolicyProviderForTest;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -33,7 +33,7 @@ import com.google.common.io.Resources;
 
 public class TestExportImportPrivileges extends AbstractTestWithStaticConfiguration {
   private File dataFile;
-  private PolicyFile policyFile;
+  private PolicyProviderForTest policyFile;
 
   @Before
   public void setup() throws Exception {
@@ -42,7 +42,7 @@ public class TestExportImportPrivileges extends AbstractTestWithStaticConfigurat
     FileOutputStream to = new FileOutputStream(dataFile);
     Resources.copy(Resources.getResource(SINGLE_TYPE_DATA_FILE_NAME), to);
     to.close();
-    policyFile = PolicyFile.setAdminOnServer1(ADMINGROUP);
+    policyFile = PolicyProviderForTest.setAdminOnServer1(ADMINGROUP);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestMetadataObjectRetrieval.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestMetadataObjectRetrieval.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestMetadataObjectRetrieval.java
index 8657b04..b10c0c2 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestMetadataObjectRetrieval.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestMetadataObjectRetrieval.java
@@ -26,7 +26,7 @@ import java.sql.Connection;
 import java.sql.ResultSet;
 import java.sql.Statement;
 
-import org.apache.sentry.provider.file.PolicyFile;
+import org.apache.sentry.tests.e2e.dbprovider.PolicyProviderForTest;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -34,12 +34,12 @@ import org.junit.Test;
 import com.google.common.io.Resources;
 
 public class TestMetadataObjectRetrieval extends AbstractTestWithStaticConfiguration {
-  private PolicyFile policyFile;
+  private PolicyProviderForTest policyFile;
   private File dataFile;
 
   @Before
   public void setup() throws Exception {
-    policyFile = PolicyFile.setAdminOnServer1(ADMINGROUP);
+    policyFile = PolicyProviderForTest.setAdminOnServer1(ADMINGROUP);
     context = createContext();
     dataFile = new File(dataDir, SINGLE_TYPE_DATA_FILE_NAME);
     FileOutputStream to = new FileOutputStream(dataFile);
@@ -458,4 +458,4 @@ public class TestMetadataObjectRetrieval extends AbstractTestWithStaticConfigura
     assertTrue(rs.next());
     assertEquals("dt=3", rs.getString(1).trim());
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestMetadataPermissions.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestMetadataPermissions.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestMetadataPermissions.java
index ad50bf0..e0642e7 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestMetadataPermissions.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestMetadataPermissions.java
@@ -21,19 +21,20 @@ import java.sql.Statement;
 
 import junit.framework.Assert;
 
-import org.apache.sentry.provider.file.PolicyFile;
+import org.apache.sentry.tests.e2e.dbprovider.PolicyProviderForTest;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
 
 public class TestMetadataPermissions extends AbstractTestWithStaticConfiguration {
-  private PolicyFile policyFile;
+  private Context context;
+  private PolicyProviderForTest policyFile;
 
   @Before
   public void setup() throws Exception {
     context = createContext();
-    policyFile = PolicyFile.setAdminOnServer1(ADMINGROUP);
+    policyFile = PolicyProviderForTest.setAdminOnServer1(ADMINGROUP);
 
     policyFile
         .addRolesToGroup(USERGROUP1, "db1_all", "db2_all")

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestMovingToProduction.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestMovingToProduction.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestMovingToProduction.java
index d5061b7..0889615 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestMovingToProduction.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestMovingToProduction.java
@@ -26,7 +26,7 @@ import java.sql.Connection;
 import java.sql.ResultSet;
 import java.sql.Statement;
 
-import org.apache.sentry.provider.file.PolicyFile;
+import org.apache.sentry.tests.e2e.dbprovider.PolicyProviderForTest;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -35,7 +35,7 @@ import com.google.common.io.Resources;
 
 public class TestMovingToProduction extends AbstractTestWithStaticConfiguration {
   private final String SINGLE_TYPE_DATA_FILE_NAME = "kv1.dat";
-  private PolicyFile policyFile;
+  private PolicyProviderForTest policyFile;
 
 
   @Before
@@ -45,7 +45,7 @@ public class TestMovingToProduction extends AbstractTestWithStaticConfiguration
     FileOutputStream to = new FileOutputStream(dataFile);
     Resources.copy(Resources.getResource(SINGLE_TYPE_DATA_FILE_NAME), to);
     to.close();
-    policyFile = PolicyFile.setAdminOnServer1(ADMINGROUP);
+    policyFile = PolicyProviderForTest.setAdminOnServer1(ADMINGROUP);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtDatabaseScope.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtDatabaseScope.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtDatabaseScope.java
index 75979fb..1e93ec6 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtDatabaseScope.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtDatabaseScope.java
@@ -31,7 +31,7 @@ import java.util.Map;
 
 import junit.framework.Assert;
 
-import org.apache.sentry.provider.file.PolicyFile;
+import org.apache.sentry.tests.e2e.dbprovider.PolicyProviderForTest;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -43,7 +43,8 @@ import com.google.common.io.Resources;
 
 public class TestPrivilegesAtDatabaseScope extends AbstractTestWithStaticConfiguration {
 
-  private PolicyFile policyFile;
+  private Context context;
+  private PolicyProviderForTest policyFile;
 
   Map <String, String >testProperties;
   private static final String SINGLE_TYPE_DATA_FILE_NAME = "kv1.dat";
@@ -52,7 +53,7 @@ public class TestPrivilegesAtDatabaseScope extends AbstractTestWithStaticConfigu
   public void setup() throws Exception {
     testProperties = new HashMap<String, String>();
 
-    policyFile = PolicyFile.setAdminOnServer1(ADMINGROUP);
+    policyFile = PolicyProviderForTest.setAdminOnServer1(ADMINGROUP);
     context = createContext();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtFunctionScope.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtFunctionScope.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtFunctionScope.java
index e66bb15..2c9924b 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtFunctionScope.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtFunctionScope.java
@@ -26,7 +26,7 @@ import java.sql.Connection;
 import java.sql.SQLException;
 import java.sql.Statement;
 
-import org.apache.sentry.provider.file.PolicyFile;
+import org.apache.sentry.tests.e2e.dbprovider.PolicyProviderForTest;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -37,7 +37,7 @@ public class TestPrivilegesAtFunctionScope extends AbstractTestWithStaticConfigu
   private final String SINGLE_TYPE_DATA_FILE_NAME = "kv1.dat";
   private File dataDir;
   private File dataFile;
-  private PolicyFile policyFile;
+  private PolicyProviderForTest policyFile;
 
   @Before
   public void setup() throws Exception {
@@ -47,7 +47,7 @@ public class TestPrivilegesAtFunctionScope extends AbstractTestWithStaticConfigu
     FileOutputStream to = new FileOutputStream(dataFile);
     Resources.copy(Resources.getResource(SINGLE_TYPE_DATA_FILE_NAME), to);
     to.close();
-    policyFile = PolicyFile.setAdminOnServer1(ADMINGROUP);
+    policyFile = PolicyProviderForTest.setAdminOnServer1(ADMINGROUP);
 
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtTableScope.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtTableScope.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtTableScope.java
index 35afc6f..6b493c4 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtTableScope.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtTableScope.java
@@ -30,7 +30,7 @@ import java.sql.Statement;
 
 import junit.framework.Assert;
 
-import org.apache.sentry.provider.file.PolicyFile;
+import org.apache.sentry.tests.e2e.dbprovider.PolicyProviderForTest;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -42,7 +42,8 @@ import com.google.common.io.Resources;
 
 public class TestPrivilegesAtTableScope extends AbstractTestWithStaticConfiguration {
 
-  private PolicyFile policyFile;
+  private Context context;
+  private PolicyProviderForTest policyFile;
 
   private final String SINGLE_TYPE_DATA_FILE_NAME = "kv1.dat";
   private final String MULTI_TYPE_DATA_FILE_NAME = "emp.dat";
@@ -50,7 +51,7 @@ public class TestPrivilegesAtTableScope extends AbstractTestWithStaticConfigurat
   @Before
   public void setup() throws Exception {
     context = createContext();
-    policyFile = PolicyFile.setAdminOnServer1(ADMINGROUP);
+    policyFile = PolicyProviderForTest.setAdminOnServer1(ADMINGROUP);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestRuntimeMetadataRetrieval.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestRuntimeMetadataRetrieval.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestRuntimeMetadataRetrieval.java
index 217b927..ddb1c8b 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestRuntimeMetadataRetrieval.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestRuntimeMetadataRetrieval.java
@@ -26,7 +26,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
-import org.apache.sentry.provider.file.PolicyFile;
+import org.apache.sentry.tests.e2e.dbprovider.PolicyProviderForTest;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -40,7 +40,8 @@ import com.google.common.io.Resources;
  * statements are validaed via a runtime fetch hook
  */
 public class TestRuntimeMetadataRetrieval extends AbstractTestWithStaticConfiguration {
-  private PolicyFile policyFile;
+  private Context context;
+  private PolicyProviderForTest policyFile;
   private final String SINGLE_TYPE_DATA_FILE_NAME = "kv1.dat";
   private File dataDir;
   private File dataFile;
@@ -53,7 +54,7 @@ public class TestRuntimeMetadataRetrieval extends AbstractTestWithStaticConfigur
     FileOutputStream to = new FileOutputStream(dataFile);
     Resources.copy(Resources.getResource(SINGLE_TYPE_DATA_FILE_NAME), to);
     to.close();
-    policyFile = PolicyFile.setAdminOnServer1(ADMINGROUP);
+    policyFile = PolicyProviderForTest.setAdminOnServer1(ADMINGROUP);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestSandboxOps.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestSandboxOps.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestSandboxOps.java
index 62db36a..10c7b82 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestSandboxOps.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestSandboxOps.java
@@ -29,6 +29,7 @@ import java.sql.Statement;
 import org.apache.hadoop.fs.Path;
 import org.apache.sentry.provider.file.PolicyFile;
 import org.apache.sentry.provider.file.PolicyFiles;
+import org.apache.sentry.tests.e2e.dbprovider.PolicyProviderForTest;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -36,7 +37,7 @@ import org.junit.Test;
 import com.google.common.io.Resources;
 
 public class TestSandboxOps  extends AbstractTestWithStaticConfiguration {
-  private PolicyFile policyFile;
+  private PolicyProviderForTest policyFile;
   private File dataFile;
   private String loadData;
   private static final String DB2_POLICY_FILE = "db2-policy-file.ini";
@@ -49,7 +50,7 @@ public class TestSandboxOps  extends AbstractTestWithStaticConfiguration {
     FileOutputStream to = new FileOutputStream(dataFile);
     Resources.copy(Resources.getResource(SINGLE_TYPE_DATA_FILE_NAME), to);
     to.close();
-    policyFile = PolicyFile.setAdminOnServer1(ADMINGROUP);
+    policyFile = PolicyProviderForTest.setAdminOnServer1(ADMINGROUP);
     loadData = "server=server1->uri=file://" + dataFile.getPath();
   }
 
@@ -59,7 +60,8 @@ public class TestSandboxOps  extends AbstractTestWithStaticConfiguration {
       context.close();
     }
   }
-  private PolicyFile addTwoUsersWithAllDb() {
+
+  private PolicyFile addTwoUsersWithAllDb() throws Exception {
     policyFile
     .addPermissionsToRole("db1_all", "server=server1->db=db1")
     .addPermissionsToRole("db2_all", "server=server1->db=db2")

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestUriPermissions.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestUriPermissions.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestUriPermissions.java
index 3aae711..2c62949 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestUriPermissions.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestUriPermissions.java
@@ -22,14 +22,15 @@ import java.sql.Statement;
 
 import junit.framework.Assert;
 
-import org.apache.sentry.provider.file.PolicyFile;
+import org.apache.sentry.tests.e2e.dbprovider.PolicyProviderForTest;
 import org.apache.sentry.tests.e2e.hive.hiveserver.HiveServerFactory;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
 public class TestUriPermissions extends AbstractTestWithStaticConfiguration {
-  private PolicyFile policyFile;
+  private Context context;
+  private PolicyProviderForTest policyFile;
 
   private static final String dataFile = "/kv1.dat";
   private String dataFilePath = this.getClass().getResource(dataFile).getFile();
@@ -37,7 +38,7 @@ public class TestUriPermissions extends AbstractTestWithStaticConfiguration {
   @Before
   public void setup() throws Exception {
     context = createContext();
-    policyFile = PolicyFile.setAdminOnServer1(ADMINGROUP);
+    policyFile = PolicyProviderForTest.setAdminOnServer1(ADMINGROUP);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/600f0f15/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestUserManagement.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestUserManagement.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestUserManagement.java
index 3fd48f8..a985e64 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestUserManagement.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestUserManagement.java
@@ -27,7 +27,7 @@ import java.sql.ResultSet;
 import java.sql.Statement;
 
 import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.sentry.provider.file.PolicyFile;
+import org.apache.sentry.tests.e2e.dbprovider.PolicyProviderForTest;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -40,7 +40,8 @@ public class TestUserManagement extends AbstractTestWithStaticConfiguration {
   private static final String tableName = "t1";
   private static final String tableComment = "Test table";
   private File dataFile;
-  private PolicyFile policyFile;
+  private Context context;
+  private PolicyProviderForTest policyFile;
 
   @Before
   public void setUp() throws Exception {
@@ -108,7 +109,7 @@ public class TestUserManagement extends AbstractTestWithStaticConfiguration {
    */
   @Test
   public void testSanity() throws Exception {
-    policyFile = PolicyFile.setAdminOnServer1(ADMINGROUP);
+    policyFile = PolicyProviderForTest.setAdminOnServer1(ADMINGROUP);
     policyFile
         .addGroupsToUser("admin1", ADMINGROUP);
     writePolicyFile(policyFile);
@@ -120,7 +121,7 @@ public class TestUserManagement extends AbstractTestWithStaticConfiguration {
    **/
   @Test
   public void testAdmin1() throws Exception {
-    policyFile = PolicyFile.setAdminOnServer1(ADMINGROUP);
+    policyFile = PolicyProviderForTest.setAdminOnServer1(ADMINGROUP);
     policyFile
         .addGroupsToUser("admin1", ADMINGROUP)
         .addGroupsToUser("admin2", ADMINGROUP)
@@ -136,7 +137,7 @@ public class TestUserManagement extends AbstractTestWithStaticConfiguration {
    **/
   @Test
   public void testAdmin3() throws Exception {
-    policyFile = PolicyFile.setAdminOnServer1(ADMINGROUP);
+    policyFile = PolicyProviderForTest.setAdminOnServer1(ADMINGROUP);
     policyFile
         .addGroupsToUser("admin1", ADMINGROUP)
         .addGroupsToUser("admin2", ADMINGROUP)
@@ -161,7 +162,7 @@ public class TestUserManagement extends AbstractTestWithStaticConfiguration {
    **/
   @Test
   public void testAdmin5() throws Exception {
-    policyFile = new PolicyFile();
+    policyFile = new PolicyProviderForTest();
     policyFile
         .addRolesToGroup("admin_group1", ADMINGROUP)
         .addRolesToGroup("admin_group2", ADMINGROUP)
@@ -178,7 +179,7 @@ public class TestUserManagement extends AbstractTestWithStaticConfiguration {
    **/
   @Test
   public void testAdmin6() throws Exception {
-    policyFile = PolicyFile.setAdminOnServer1(ADMINGROUP);
+    policyFile = PolicyProviderForTest.setAdminOnServer1(ADMINGROUP);
     policyFile
         .addGroupsToUser("admin1", ADMINGROUP)
         .addRolesToGroup("group1", "non_admin_role")
@@ -199,7 +200,7 @@ public class TestUserManagement extends AbstractTestWithStaticConfiguration {
    **/
   @Test
   public void testGroup2() throws Exception {
-    policyFile = new PolicyFile();
+    policyFile = new PolicyProviderForTest();
     policyFile
         .addRolesToGroup("group1", ADMINGROUP, "analytics")
         .addPermissionsToRole(ADMINGROUP, "server=server1")
@@ -215,7 +216,7 @@ public class TestUserManagement extends AbstractTestWithStaticConfiguration {
    **/
   @Test
   public void testGroup4() throws Exception {
-    policyFile = PolicyFile.setAdminOnServer1(ADMINGROUP);
+    policyFile = PolicyProviderForTest.setAdminOnServer1(ADMINGROUP);
     policyFile
         .addGroupsToUser("admin1", ADMINGROUP)
         .addRolesToGroup("group1", "non_admin_role", "load_data")
@@ -248,7 +249,7 @@ public class TestUserManagement extends AbstractTestWithStaticConfiguration {
   @Test
   public void testGroup5() throws Exception {
 
-    policyFile = PolicyFile.setAdminOnServer1(ADMINGROUP);
+    policyFile = PolicyProviderForTest.setAdminOnServer1(ADMINGROUP);
     policyFile
         .addGroupsToUser("admin1", ADMINGROUP)
         .addRolesToGroup("group1", "non_admin_role", "load_data")
@@ -272,7 +273,7 @@ public class TestUserManagement extends AbstractTestWithStaticConfiguration {
    **/
   @Test
   public void testGroup6() throws Exception {
-    policyFile = PolicyFile.setAdminOnServer1(ADMINGROUP);
+    policyFile = PolicyProviderForTest.setAdminOnServer1(ADMINGROUP);
     policyFile
         .addGroupsToUser("admin1", ADMINGROUP)
         .addRolesToGroup("group1~!@#$%^&*()+-", "analytics", "load_data")
@@ -296,7 +297,7 @@ public class TestUserManagement extends AbstractTestWithStaticConfiguration {
    **/
   @Test
   public void testGroup7() throws Exception {
-    policyFile = new PolicyFile();
+    policyFile = new PolicyProviderForTest();
     policyFile
         .addRolesToGroup("group1", ADMINGROUP)
         .addPermissionsToRole(ADMINGROUP, "server=server1")
@@ -312,7 +313,7 @@ public class TestUserManagement extends AbstractTestWithStaticConfiguration {
    **/
   @Test
   public void testGroup8() throws Exception {
-    policyFile = PolicyFile.setAdminOnServer1(ADMINGROUP);
+    policyFile = PolicyProviderForTest.setAdminOnServer1(ADMINGROUP);
     policyFile
         .addGroupsToUser("admin1", ADMINGROUP)
         .addRolesToGroup("group1", "analytics")