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:22 UTC

[48/50] [abbrv] incubator-sentry git commit: SENTRY-828: Cleanup the unnecessary ProviderBackend (Colin Ma, Reviewed by: Guoquan Shen)

SENTRY-828: Cleanup the unnecessary ProviderBackend (Colin Ma, Reviewed by: Guoquan Shen)


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

Branch: refs/heads/hive_plugin_v2
Commit: 35c62ffc6614ad97ecc537bd48da0cbf123269cc
Parents: 1712142
Author: Colin Ma <co...@apache.org>
Authored: Wed Aug 12 08:35:21 2015 +0800
Committer: Colin Ma <co...@apache.org>
Committed: Wed Aug 12 08:35:21 2015 +0800

----------------------------------------------------------------------
 .../binding/solr/authz/SolrAuthzBinding.java    |  50 +++++-
 .../sentry/sqoop/binding/SqoopAuthBinding.java  |  20 ++-
 .../sqoop/binding/SqoopProviderBackend.java     |  44 -----
 .../apache/sentry/sqoop/conf/SqoopAuthConf.java |   5 +-
 .../core/model/search/SearchConstants.java      |   2 +-
 .../generic/SentryGenericProviderBackend.java   |  40 +++--
 .../thrift/SearchPolicyServiceClient.java       | 159 -------------------
 .../service/thrift/SearchProviderBackend.java   | 141 ----------------
 .../AbstractSolrSentryTestWithDbProvider.java   |  64 ++++++--
 .../db/integration/TestSolrAdminOperations.java |  71 +++++----
 .../integration/TestSolrDocLevelOperations.java |  10 +-
 .../db/integration/TestSolrQueryOperations.java |  21 +--
 .../integration/TestSolrUpdateOperations.java   |  14 +-
 .../e2e/sqoop/AbstractSqoopSentryTestBase.java  |  14 +-
 14 files changed, 206 insertions(+), 449 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/35c62ffc/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/authz/SolrAuthzBinding.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/authz/SolrAuthzBinding.java b/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/authz/SolrAuthzBinding.java
index 7f59eaa..2accbbf 100644
--- a/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/authz/SolrAuthzBinding.java
+++ b/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/authz/SolrAuthzBinding.java
@@ -16,34 +16,45 @@
  */
 package org.apache.sentry.binding.solr.authz;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
+import static org.apache.sentry.core.model.search.SearchConstants.SENTRY_SEARCH_CLUSTER_DEFAULT;
+import static org.apache.sentry.core.model.search.SearchConstants.SENTRY_SEARCH_CLUSTER_KEY;
+import static org.apache.sentry.core.model.search.SearchModelAuthorizable.AuthorizableType.Collection;
+
 import java.io.File;
 import java.io.IOException;
 import java.lang.reflect.Constructor;
 import java.util.Arrays;
+import java.util.List;
 import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
-import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.sentry.SentryUserException;
 import org.apache.sentry.binding.solr.conf.SolrAuthzConf;
 import org.apache.sentry.binding.solr.conf.SolrAuthzConf.AuthzConfVars;
+import org.apache.sentry.core.common.Action;
 import org.apache.sentry.core.common.ActiveRoleSet;
 import org.apache.sentry.core.common.Subject;
 import org.apache.sentry.core.model.search.Collection;
 import org.apache.sentry.core.model.search.SearchModelAction;
 import org.apache.sentry.policy.common.PolicyEngine;
+import org.apache.sentry.provider.common.AuthorizationComponent;
 import org.apache.sentry.provider.common.AuthorizationProvider;
 import org.apache.sentry.provider.common.GroupMappingService;
 import org.apache.sentry.provider.common.HadoopGroupResourceAuthorizationProvider;
 import org.apache.sentry.provider.common.ProviderBackend;
-import org.apache.sentry.provider.db.generic.service.thrift.SearchPolicyServiceClient;
-import org.apache.sentry.provider.db.generic.service.thrift.SearchProviderBackend;
+import org.apache.sentry.provider.db.generic.SentryGenericProviderBackend;
+import org.apache.sentry.provider.db.generic.service.thrift.SentryGenericServiceClient;
+import org.apache.sentry.provider.db.generic.service.thrift.TAuthorizable;
+import org.apache.sentry.provider.db.generic.service.thrift.TSentryGrantOption;
+import org.apache.sentry.provider.db.generic.service.thrift.TSentryPrivilege;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Strings;
+import com.google.common.collect.Lists;
 
 public class SolrAuthzBinding {
   private static final Logger LOG = LoggerFactory
@@ -85,6 +96,7 @@ public class SolrAuthzBinding {
       authzConf.get(AuthzConfVars.AUTHZ_PROVIDER_BACKEND.getVar());
     String policyEngineName =
       authzConf.get(AuthzConfVars.AUTHZ_POLICY_ENGINE.getVar());
+    String serviceName = authzConf.get(SENTRY_SEARCH_CLUSTER_KEY, SENTRY_SEARCH_CLUSTER_DEFAULT);
 
     LOG.debug("Using authorization provider " + authProviderName +
       " with resource " + resourceName + ", policy engine "
@@ -97,6 +109,13 @@ public class SolrAuthzBinding {
       // we don't use kerberos, for testing
       UserGroupInformation.setConfiguration(authzConf);
     }
+
+    // the SearchProviderBackend is deleted in SENTRY-828, this is for the compatible with the
+    // previous Sentry.
+    if ("org.apache.sentry.provider.db.generic.service.thrift.SearchProviderBackend"
+        .equals(providerBackendName)) {
+      providerBackendName = SentryGenericProviderBackend.class.getName();
+    }
     Constructor<?> providerBackendConstructor =
       Class.forName(providerBackendName).getDeclaredConstructor(Configuration.class, String.class);
     providerBackendConstructor.setAccessible(true);
@@ -104,6 +123,12 @@ public class SolrAuthzBinding {
     providerBackend =
       (ProviderBackend) providerBackendConstructor.newInstance(new Object[] {authzConf, resourceName});
 
+    if (providerBackend instanceof SentryGenericProviderBackend) {
+      ((SentryGenericProviderBackend) providerBackend)
+          .setComponentType(AuthorizationComponent.Search);
+      ((SentryGenericProviderBackend) providerBackend).setServiceName(serviceName);
+    }
+
     // load the policy engine class
     Constructor<?> policyConstructor =
       Class.forName(policyEngineName).getDeclaredConstructor(ProviderBackend.class);
@@ -232,11 +257,11 @@ public class SolrAuthzBinding {
    * If the binding uses the searchProviderBackend, it can sync privilege with Sentry Service
    */
   public boolean isSyncEnabled() {
-    return (providerBackend instanceof SearchProviderBackend);
+    return (providerBackend instanceof SentryGenericProviderBackend);
   }
 
-  public SearchPolicyServiceClient getClient() throws Exception {
-    return new SearchPolicyServiceClient(authzConf);
+  public SentryGenericServiceClient getClient() throws Exception {
+    return new SentryGenericServiceClient(authzConf);
   }
 
   /**
@@ -248,10 +273,19 @@ public class SolrAuthzBinding {
     if (!isSyncEnabled()) {
       return;
     }
-    SearchPolicyServiceClient client = null;
+    SentryGenericServiceClient client = null;
     try {
       client = getClient();
-      client.dropCollectionPrivilege(collection, bindingSubject.getName());
+      TSentryPrivilege tPrivilege = new TSentryPrivilege();
+      tPrivilege.setComponent(AuthorizationComponent.Search);
+      tPrivilege.setServiceName(authzConf.get(SENTRY_SEARCH_CLUSTER_KEY,
+          SENTRY_SEARCH_CLUSTER_DEFAULT));
+      tPrivilege.setAction(Action.ALL);
+      tPrivilege.setGrantOption(TSentryGrantOption.UNSET);
+      List<TAuthorizable> authorizables = Lists.newArrayList(new TAuthorizable(Collection.name(),
+          collection));
+      tPrivilege.setAuthorizables(authorizables);
+      client.dropPrivilege(bindingSubject.getName(), AuthorizationComponent.Search, tPrivilege);
     } catch (SentryUserException ex) {
       throw new SentrySolrAuthorizationException("User " + bindingSubject.getName() +
           " can't delete privileges for collection " + collection);

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/35c62ffc/sentry-binding/sentry-binding-sqoop/src/main/java/org/apache/sentry/sqoop/binding/SqoopAuthBinding.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-sqoop/src/main/java/org/apache/sentry/sqoop/binding/SqoopAuthBinding.java b/sentry-binding/sentry-binding-sqoop/src/main/java/org/apache/sentry/sqoop/binding/SqoopAuthBinding.java
index 4052e2a..ee0fbfa 100644
--- a/sentry-binding/sentry-binding-sqoop/src/main/java/org/apache/sentry/sqoop/binding/SqoopAuthBinding.java
+++ b/sentry-binding/sentry-binding-sqoop/src/main/java/org/apache/sentry/sqoop/binding/SqoopAuthBinding.java
@@ -33,6 +33,7 @@ import org.apache.sentry.policy.common.PolicyEngine;
 import org.apache.sentry.provider.common.AuthorizationComponent;
 import org.apache.sentry.provider.common.AuthorizationProvider;
 import org.apache.sentry.provider.common.ProviderBackend;
+import org.apache.sentry.provider.db.generic.SentryGenericProviderBackend;
 import org.apache.sentry.provider.db.generic.service.thrift.SentryGenericServiceClient;
 import org.apache.sentry.provider.db.generic.service.thrift.TAuthorizable;
 import org.apache.sentry.provider.db.generic.service.thrift.TSentryGrantOption;
@@ -84,18 +85,29 @@ public class SqoopAuthBinding {
     String resourceName = authConf.get(AuthzConfVars.AUTHZ_PROVIDER_RESOURCE.getVar(), AuthzConfVars.AUTHZ_PROVIDER_RESOURCE.getDefault());
     String providerBackendName = authConf.get(AuthzConfVars.AUTHZ_PROVIDER_BACKEND.getVar(), AuthzConfVars.AUTHZ_PROVIDER_BACKEND.getDefault());
     String policyEngineName = authConf.get(AuthzConfVars.AUTHZ_POLICY_ENGINE.getVar(), AuthzConfVars.AUTHZ_POLICY_ENGINE.getDefault());
+    String serviceName = authConf.get(AuthzConfVars.AUTHZ_SERVER_NAME.getVar());
     if (LOG.isDebugEnabled()) {
       LOG.debug("Using authorization provider " + authProviderName +
           " with resource " + resourceName + ", policy engine "
           + policyEngineName + ", provider backend " + providerBackendName);
     }
 
+    // the SqoopProviderBackend is deleted in SENTRY-828, this is for the compatible with the
+    // previous Sentry.
+    if ("org.apache.sentry.sqoop.binding.SqoopProviderBackend".equals(providerBackendName)) {
+      providerBackendName = SentryGenericProviderBackend.class.getName();
+    }
+
     //Instantiate the configured providerBackend
-    Constructor<?> providerBackendConstructor =
-        Class.forName(providerBackendName).getDeclaredConstructor(Configuration.class, String.class);
+    Constructor<?> providerBackendConstructor = Class.forName(providerBackendName)
+        .getDeclaredConstructor(Configuration.class, String.class);
     providerBackendConstructor.setAccessible(true);
-    providerBackend =
-          (ProviderBackend) providerBackendConstructor.newInstance(new Object[] {authConf, resourceName});
+    providerBackend = (ProviderBackend) providerBackendConstructor.newInstance(new Object[] {
+        authConf, resourceName });
+    if (providerBackend instanceof SentryGenericProviderBackend) {
+      ((SentryGenericProviderBackend) providerBackend).setComponentType(COMPONENT_TYPE);
+      ((SentryGenericProviderBackend) providerBackend).setServiceName(serviceName);
+    }
 
     //Instantiate the configured policyEngine
     Constructor<?> policyConstructor =

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/35c62ffc/sentry-binding/sentry-binding-sqoop/src/main/java/org/apache/sentry/sqoop/binding/SqoopProviderBackend.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-sqoop/src/main/java/org/apache/sentry/sqoop/binding/SqoopProviderBackend.java b/sentry-binding/sentry-binding-sqoop/src/main/java/org/apache/sentry/sqoop/binding/SqoopProviderBackend.java
deleted file mode 100644
index cadc2f5..0000000
--- a/sentry-binding/sentry-binding-sqoop/src/main/java/org/apache/sentry/sqoop/binding/SqoopProviderBackend.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.sentry.sqoop.binding;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.sentry.core.model.sqoop.Server;
-import org.apache.sentry.provider.common.AuthorizationComponent;
-import org.apache.sentry.provider.db.generic.SentryGenericProviderBackend;
-import org.apache.sentry.sqoop.conf.SqoopAuthConf.AuthzConfVars;
-
-public class SqoopProviderBackend extends SentryGenericProviderBackend {
-  private Server sqoopServer;
-  public SqoopProviderBackend(Configuration conf, String resourcePath) throws Exception {
-    super(conf);
-    sqoopServer = new Server(conf.get(AuthzConfVars.AUTHZ_SERVER_NAME.getVar()));
-  }
-  @Override
-  public String getComponentType() {
-    return AuthorizationComponent.SQOOP;
-  }
-
-  /**
-   * SqoopProviderBackend use the name of Sqoop Server as the identifier to
-   * distinguish itself from multiple Sqoop Servers
-   */
-  @Override
-  public String getComponentIdentifier() {
-    return sqoopServer.getName();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/35c62ffc/sentry-binding/sentry-binding-sqoop/src/main/java/org/apache/sentry/sqoop/conf/SqoopAuthConf.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-sqoop/src/main/java/org/apache/sentry/sqoop/conf/SqoopAuthConf.java b/sentry-binding/sentry-binding-sqoop/src/main/java/org/apache/sentry/sqoop/conf/SqoopAuthConf.java
index fcf7860..097e7f7 100644
--- a/sentry-binding/sentry-binding-sqoop/src/main/java/org/apache/sentry/sqoop/conf/SqoopAuthConf.java
+++ b/sentry-binding/sentry-binding-sqoop/src/main/java/org/apache/sentry/sqoop/conf/SqoopAuthConf.java
@@ -17,6 +17,7 @@
 package org.apache.sentry.sqoop.conf;
 
 import java.net.URL;
+
 import org.apache.hadoop.conf.Configuration;
 
 public class SqoopAuthConf extends Configuration {
@@ -30,7 +31,9 @@ public class SqoopAuthConf extends Configuration {
   public static enum AuthzConfVars {
     AUTHZ_PROVIDER("sentry.sqoop.provider","org.apache.sentry.provider.common.HadoopGroupResourceAuthorizationProvider"),
     AUTHZ_PROVIDER_RESOURCE("sentry.sqoop.provider.resource", ""),
-    AUTHZ_PROVIDER_BACKEND("sentry.sqoop.provider.backend","org.apache.sentry.provider.file.SimpleFileProviderBackend"),
+    AUTHZ_PROVIDER_BACKEND(
+        "sentry.sqoop.provider.backend",
+        "org.apache.sentry.provider.db.generic.SentryGenericProviderBackend"),
     AUTHZ_POLICY_ENGINE("sentry.sqoop.policy.engine","org.apache.sentry.policy.sqoop.SimpleSqoopPolicyEngine"),
     AUTHZ_SERVER_NAME("sentry.sqoop.name", ""),
     AUTHZ_TESTING_MODE("sentry.sqoop.testing.mode", "false");

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/35c62ffc/sentry-core/sentry-core-model-search/src/main/java/org/apache/sentry/core/model/search/SearchConstants.java
----------------------------------------------------------------------
diff --git a/sentry-core/sentry-core-model-search/src/main/java/org/apache/sentry/core/model/search/SearchConstants.java b/sentry-core/sentry-core-model-search/src/main/java/org/apache/sentry/core/model/search/SearchConstants.java
index 16b9195..36f5b21 100644
--- a/sentry-core/sentry-core-model-search/src/main/java/org/apache/sentry/core/model/search/SearchConstants.java
+++ b/sentry-core/sentry-core-model-search/src/main/java/org/apache/sentry/core/model/search/SearchConstants.java
@@ -27,5 +27,5 @@ public class SearchConstants {
    * sentry.search.cluster=cluster1 or cluster2 to communicate with sentry service for authorization
    */
   public static final String SENTRY_SEARCH_CLUSTER_KEY = "sentry.search.cluster";
-  public static final String SENTRY_SEARCH_CLUSTER_DEFAULT = "clutser1";
+  public static final String SENTRY_SEARCH_CLUSTER_DEFAULT = "cluster1";
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/35c62ffc/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/SentryGenericProviderBackend.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/SentryGenericProviderBackend.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/SentryGenericProviderBackend.java
index 11ffde2..50edeb3 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/SentryGenericProviderBackend.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/SentryGenericProviderBackend.java
@@ -39,12 +39,17 @@ import com.google.common.collect.Sets;
 /**
  * This class used when any component such as Hive, Solr or Sqoop want to integration with the Sentry service
  */
-public abstract class SentryGenericProviderBackend implements ProviderBackend {
+public class SentryGenericProviderBackend implements ProviderBackend {
   private static final Logger LOGGER = LoggerFactory.getLogger(SentryGenericProviderBackend.class);
   private final Configuration conf;
   private volatile boolean initialized = false;
+  private String componentType;
+  private String serviceName;
 
-  public SentryGenericProviderBackend(Configuration conf) throws Exception {
+  // ProviderBackend should have the same construct to support the reflect in authBinding,
+  // eg:SqoopAuthBinding
+  public SentryGenericProviderBackend(Configuration conf, String resource)
+      throws Exception {
     this.conf = conf;
   }
 
@@ -73,9 +78,8 @@ public abstract class SentryGenericProviderBackend implements ProviderBackend {
     SentryGenericServiceClient client = null;
     try {
       client = getClient();
-      return ImmutableSet.copyOf(client.listPrivilegesForProvider(
-          getComponentType(), getComponentIdentifier(), roleSet, groups,
-          Arrays.asList(authorizableHierarchy)));
+      return ImmutableSet.copyOf(client.listPrivilegesForProvider(componentType, serviceName,
+          roleSet, groups, Arrays.asList(authorizableHierarchy)));
     } catch (SentryUserException e) {
       String msg = "Unable to obtain privileges from server: " + e.getMessage();
       LOGGER.error(msg, e);
@@ -138,16 +142,20 @@ public abstract class SentryGenericProviderBackend implements ProviderBackend {
   public void close() {
   }
 
-  /**
-   * Get the component type for the Generic Provider backend, such as Hive,Solr or Sqoop
-   */
-  public abstract String getComponentType();
+  public void setComponentType(String componentType) {
+    this.componentType = componentType;
+  }
+
+  public String getComponentType() {
+    return componentType;
+  }
+
+  public String getServiceName() {
+    return serviceName;
+  }
+
+  public void setServiceName(String serviceName) {
+    this.serviceName = serviceName;
+  }
 
-  /**
-   * When the providerBackend want to get privileges from the Sentry service.
-   * The component identifier is very important to Sentry service. Take the component type is Hive for example,
-   * when there are multiple HiveServers implemented role-based authorization via Sentry. Each HiveServer must uses a
-   * identifier to distinguish itself from multiple HiveServers.
-   */
-  public abstract String getComponentIdentifier();
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/35c62ffc/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/thrift/SearchPolicyServiceClient.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/thrift/SearchPolicyServiceClient.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/thrift/SearchPolicyServiceClient.java
deleted file mode 100644
index 1ed3fcd..0000000
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/thrift/SearchPolicyServiceClient.java
+++ /dev/null
@@ -1,159 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.sentry.provider.db.generic.service.thrift;
-
-import java.util.List;
-import java.util.Set;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.sentry.SentryUserException;
-import org.apache.sentry.core.common.Action;
-import org.apache.sentry.core.common.ActiveRoleSet;
-import org.apache.sentry.core.common.Authorizable;
-import org.apache.sentry.core.model.search.Collection;
-import org.apache.sentry.provider.common.AuthorizationComponent;
-
-import com.google.common.collect.Lists;
-
-import static org.apache.sentry.core.model.search.SearchModelAuthorizable.AuthorizableType.Collection;
-import static org.apache.sentry.core.model.search.SearchConstants.SENTRY_SEARCH_CLUSTER_KEY;
-import static org.apache.sentry.core.model.search.SearchConstants.SENTRY_SEARCH_CLUSTER_DEFAULT;
-
-/**
- * This search policy client will be used in the solr component to communicate with Sentry service.
- *
- */
-public class SearchPolicyServiceClient {
-  private static final String COMPONENT_TYPE = AuthorizationComponent.Search;
-
-  private String searchClusterName;
-  private SentryGenericServiceClient client;
-
-  public SearchPolicyServiceClient(Configuration conf) throws Exception {
-    this.searchClusterName = conf.get(SENTRY_SEARCH_CLUSTER_KEY, SENTRY_SEARCH_CLUSTER_DEFAULT);
-    this.client = new SentryGenericServiceClient(conf);
-  }
-
-  public void createRole(final String requestor, final String roleName)
-      throws SentryUserException {
-    client.createRole(requestor, roleName, COMPONENT_TYPE);
-  }
-
-  public void createRoleIfNotExist(final String requestor,
-      final String roleName) throws SentryUserException {
-    client.createRoleIfNotExist(requestor, roleName, COMPONENT_TYPE);
-  }
-
-  public void dropRole(final String requestor, final String roleName)
-      throws SentryUserException {
-    client.dropRole(requestor, roleName, COMPONENT_TYPE);
-  }
-
-  public void dropRoleIfExists(final String requestor, final String roleName)
-      throws SentryUserException {
-    client.dropRoleIfExists(requestor, roleName, COMPONENT_TYPE);
-  }
-
-  public void addRoleToGroups(final String requestor, final String roleName,
-      final Set<String> groups) throws SentryUserException {
-    client.addRoleToGroups(requestor, roleName, COMPONENT_TYPE, groups);
-  }
-
-  public void deleteRoleFromGroups(final String requestor, final String roleName,
-      final Set<String> groups) throws SentryUserException {
-    client.deleteRoleToGroups(requestor, roleName, COMPONENT_TYPE, groups);
-  }
-
-  public void grantCollectionPrivilege(final String collection, final String requestor,
-      final String roleName,final String action) throws SentryUserException {
-    grantCollectionPrivilege(collection, requestor, roleName, action, false);
-  }
-
-  public void grantCollectionPrivilege(final String collection, final String requestor,
-      final String roleName, final String action, final Boolean grantOption) throws SentryUserException {
-    TSentryPrivilege tPrivilege = toTSentryPrivilege(collection, action, grantOption);
-    client.grantPrivilege(requestor, roleName, COMPONENT_TYPE, tPrivilege);
-  }
-
-  public void revokeCollectionPrivilege(final String collection, final String requestor, final String roleName,
-      final String action) throws SentryUserException {
-    revokeCollectionPrivilege(collection, requestor, roleName, action, false);
-  }
-
-  public void revokeCollectionPrivilege(final String collection, final String requestor, final String roleName,
-      final String action, final Boolean grantOption) throws SentryUserException {
-    TSentryPrivilege tPrivilege = toTSentryPrivilege(collection, action, grantOption);
-    client.revokePrivilege(requestor, roleName, COMPONENT_TYPE, tPrivilege);
-  }
-
-  public void renameCollectionPrivilege(final String oldCollection, final String newCollection, final String requestor)
-      throws SentryUserException {
-    client.renamePrivilege(requestor, COMPONENT_TYPE, searchClusterName, Lists.newArrayList(new Collection(oldCollection)),
-        Lists.newArrayList(new Collection(newCollection)));
-  }
-
-  public void dropCollectionPrivilege(final String collection, final String requestor) throws SentryUserException {
-    final TSentryPrivilege tPrivilege = toTSentryPrivilege(collection, Action.ALL, null);
-    client.dropPrivilege(requestor, COMPONENT_TYPE, tPrivilege);
-  }
-
-  public Set<TSentryRole> listAllRoles(final String user) throws SentryUserException {
-    return client.listAllRoles(user, COMPONENT_TYPE);
-  }
-
-  public Set<TSentryRole> listRolesByGroupName(final String requestor, final String groupName) throws SentryUserException {
-    return client.listRolesByGroupName(requestor, groupName, COMPONENT_TYPE);
-  }
-
-  public Set<TSentryPrivilege> listPrivilegesByRoleName(
-      final String requestor, final String roleName,
-      final List<? extends Authorizable> authorizables) throws SentryUserException {
-    return client.listPrivilegesByRoleName(requestor, roleName, COMPONENT_TYPE, searchClusterName, authorizables);
-  }
-
-  public Set<String> listPrivilegesForProvider(final ActiveRoleSet roleSet, final Set<String> groups,
-      final List<? extends Authorizable> authorizables) throws SentryUserException {
-    return client.listPrivilegesForProvider(COMPONENT_TYPE, searchClusterName, roleSet, groups, authorizables);
-  }
-
-  private TSentryPrivilege toTSentryPrivilege(String collection, String action,
-      Boolean grantOption) {
-    TSentryPrivilege tPrivilege = new TSentryPrivilege();
-    tPrivilege.setComponent(COMPONENT_TYPE);
-    tPrivilege.setServiceName(searchClusterName);
-    tPrivilege.setAction(action);
-
-    if (grantOption == null) {
-      tPrivilege.setGrantOption(TSentryGrantOption.UNSET);
-    } else if (grantOption) {
-      tPrivilege.setGrantOption(TSentryGrantOption.TRUE);
-    } else {
-      tPrivilege.setGrantOption(TSentryGrantOption.FALSE);
-    }
-
-    List<TAuthorizable> authorizables = Lists.newArrayList(new TAuthorizable(Collection.name(), collection));
-    tPrivilege.setAuthorizables(authorizables);
-    return tPrivilege;
-  }
-
-  public void close() {
-    if (client != null) {
-      client.close();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/35c62ffc/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/thrift/SearchProviderBackend.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/thrift/SearchProviderBackend.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/thrift/SearchProviderBackend.java
deleted file mode 100644
index ae324bf..0000000
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/thrift/SearchProviderBackend.java
+++ /dev/null
@@ -1,141 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.sentry.provider.db.generic.service.thrift;
-
-import java.util.Arrays;
-import java.util.Set;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.sentry.SentryUserException;
-import org.apache.sentry.core.common.ActiveRoleSet;
-import org.apache.sentry.core.common.Authorizable;
-import org.apache.sentry.core.common.SentryConfigurationException;
-import org.apache.sentry.core.common.Subject;
-import org.apache.sentry.provider.common.ProviderBackend;
-import org.apache.sentry.provider.common.ProviderBackendContext;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Sets;
-
-/**
- * when Solr integration with Database store, this backend will communicate with Sentry service to get
- * privileges according to the requested groups
- *
- */
-public class SearchProviderBackend implements ProviderBackend {
-  private static final Logger LOGGER = LoggerFactory.getLogger(SearchProviderBackend.class);
-  private final Configuration conf;
-  private final Subject subject;
-  private volatile boolean initialized = false;
-
-  public SearchProviderBackend(Configuration conf, String resourcePath) throws Exception {
-    this.conf = conf;
-    /**
-     * Who create the searchProviderBackend, this subject will been used the requester to communicate
-     * with Sentry Service
-     */
-    subject = new Subject(UserGroupInformation.getCurrentUser()
-        .getShortUserName());
-  }
-
-  @Override
-  public void initialize(ProviderBackendContext context) {
-    if (initialized) {
-      throw new IllegalStateException("SearchProviderBackend has already been initialized, cannot be initialized twice");
-    }
-    this.initialized = true;
-  }
-
-  @Override
-  public ImmutableSet<String> getPrivileges(Set<String> groups,
-      ActiveRoleSet roleSet, Authorizable... authorizableHierarchy) {
-    if (!initialized) {
-      throw new IllegalStateException("SearchProviderBackend has not been properly initialized");
-    }
-    SearchPolicyServiceClient client = null;
-    try {
-      client = getClient();
-      return ImmutableSet.copyOf(client.listPrivilegesForProvider(roleSet, groups, Arrays.asList(authorizableHierarchy)));
-    } catch (SentryUserException e) {
-      String msg = "Unable to obtain privileges from server: " + e.getMessage();
-      LOGGER.error(msg, e);
-    } catch (Exception e) {
-      String msg = "Unable to obtain client:" + e.getMessage();
-      LOGGER.error(msg, e);
-    } finally {
-      if (client != null) {
-        client.close();
-      }
-    }
-    return ImmutableSet.of();
-  }
-
-  @Override
-  public ImmutableSet<String> getRoles(Set<String> groups, ActiveRoleSet roleSet) {
-    if (!initialized) {
-      throw new IllegalStateException("SearchProviderBackend has not been properly initialized");
-    }
-    SearchPolicyServiceClient client = null;
-    try {
-      Set<TSentryRole> tRoles = Sets.newHashSet();
-      client = getClient();
-      //get the roles according to group
-      for (String group : groups) {
-        tRoles.addAll(client.listRolesByGroupName(subject.getName(), group));
-      }
-      Set<String> roles = Sets.newHashSet();
-      for (TSentryRole tRole : tRoles) {
-        roles.add(tRole.getRoleName());
-      }
-      return ImmutableSet.copyOf(roleSet.isAll() ? roles : Sets.intersection(roles, roleSet.getRoles()));
-    } catch (SentryUserException e) {
-      String msg = "Unable to obtain roles from server: " + e.getMessage();
-      LOGGER.error(msg, e);
-    } catch (Exception e) {
-      String msg = "Unable to obtain client:" + e.getMessage();
-      LOGGER.error(msg, e);
-    } finally {
-      if (client != null) {
-        client.close();
-      }
-    }
-    return ImmutableSet.of();
-  }
-
-  public SearchPolicyServiceClient getClient() throws Exception {
-    return new SearchPolicyServiceClient(conf);
-  }
-
-  /**
-   * SearchProviderBackend does nothing in the validatePolicy()
-   */
-  @Override
-  public void validatePolicy(boolean strictValidation)
-      throws SentryConfigurationException {
-    if (!initialized) {
-      throw new IllegalStateException("Backend has not been properly initialized");
-    }
-  }
-
-  @Override
-  public void close() {
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/35c62ffc/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/AbstractSolrSentryTestWithDbProvider.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/AbstractSolrSentryTestWithDbProvider.java b/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/AbstractSolrSentryTestWithDbProvider.java
index 247abd6..33b35e6 100644
--- a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/AbstractSolrSentryTestWithDbProvider.java
+++ b/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/AbstractSolrSentryTestWithDbProvider.java
@@ -18,9 +18,12 @@
 package org.apache.sentry.tests.e2e.solr.db.integration;
 
 
+import static org.apache.sentry.core.model.search.SearchModelAuthorizable.AuthorizableType.Collection;
+
 import java.io.File;
 import java.io.FileOutputStream;
 import java.util.Comparator;
+import java.util.List;
 import java.util.TreeMap;
 import java.util.UUID;
 import java.util.concurrent.TimeoutException;
@@ -32,12 +35,16 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.sentry.SentryUserException;
 import org.apache.sentry.binding.solr.HdfsTestUtil;
 import org.apache.sentry.binding.solr.conf.SolrAuthzConf.AuthzConfVars;
+import org.apache.sentry.core.common.Action;
 import org.apache.sentry.core.model.search.SearchConstants;
-import org.apache.sentry.provider.common.AuthorizationComponent;
-import org.apache.sentry.provider.db.generic.service.thrift.SearchPolicyServiceClient;
-import org.apache.sentry.provider.db.generic.service.thrift.SearchProviderBackend;
+import org.apache.sentry.provider.db.generic.SentryGenericProviderBackend;
+import org.apache.sentry.provider.db.generic.service.thrift.SentryGenericServiceClient;
+import org.apache.sentry.provider.db.generic.service.thrift.TAuthorizable;
+import org.apache.sentry.provider.db.generic.service.thrift.TSentryGrantOption;
+import org.apache.sentry.provider.db.generic.service.thrift.TSentryPrivilege;
 import org.apache.sentry.provider.file.LocalGroupResourceAuthorizationProvider;
 import org.apache.sentry.provider.file.PolicyFile;
 import org.apache.sentry.service.thrift.SentryService;
@@ -52,6 +59,7 @@ import org.junit.BeforeClass;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
 /**
@@ -68,11 +76,13 @@ public class AbstractSolrSentryTestWithDbProvider extends AbstractSolrSentryTest
   protected static final String ADMIN_GROUP = "admin_group";
   protected static final String ADMIN_ROLE  = "admin_role";
   protected static final String ADMIN_COLLECTION_NAME = "admin";
+  protected static final String COMPONENT_SOLR = "solr";
+  protected static final String CLUSTER_NAME = SearchConstants.SENTRY_SEARCH_CLUSTER_DEFAULT;
 
   protected static final Configuration conf = new Configuration(false);
 
   protected static SentryService server;
-  protected static SearchPolicyServiceClient client;
+  protected static SentryGenericServiceClient client;
 
   protected static File baseDir;
   protected static File hdfsDir;
@@ -129,7 +139,8 @@ public class AbstractSolrSentryTestWithDbProvider extends AbstractSolrSentryTest
         ServerConfig.SENTRY_STORE_LOCAL_GROUP_MAPPING);
     conf.set(AuthzConfVars.AUTHZ_PROVIDER.getVar(),
         LocalGroupResourceAuthorizationProvider.class.getName());
-    conf.set(AuthzConfVars.AUTHZ_PROVIDER_BACKEND.getVar(), SearchProviderBackend.class.getName());
+    conf.set(AuthzConfVars.AUTHZ_PROVIDER_BACKEND.getVar(),
+        SentryGenericProviderBackend.class.getName());
     conf.set(AuthzConfVars.AUTHZ_PROVIDER_RESOURCE.getVar(), policyFilePath.getPath());
   }
 
@@ -193,7 +204,7 @@ public class AbstractSolrSentryTestWithDbProvider extends AbstractSolrSentryTest
   }
 
   public static void connectToSentryService() throws Exception {
-    client = new SearchPolicyServiceClient(conf);
+    client = new SentryGenericServiceClient(conf);
   }
 
   public static void stopAllService() throws Exception {
@@ -261,16 +272,47 @@ public class AbstractSolrSentryTestWithDbProvider extends AbstractSolrSentryTest
     writePolicyFile();
 
     for (int i = 0; i < roles.length; i++) {
-      client.createRole(ADMIN_USER, roles[i]);
-      client.addRoleToGroups(ADMIN_USER, roles[i], Sets.newHashSet(groups[i]));
+      client.createRole(ADMIN_USER, roles[i], COMPONENT_SOLR);
+      client.addRoleToGroups(ADMIN_USER, roles[i], COMPONENT_SOLR, Sets.newHashSet(groups[i]));
     }
 
     /**
      * user[admin]->group[admin]->role[admin]
      * grant ALL privilege on collection ALL to role admin
      */
-    client.createRole(ADMIN_USER, ADMIN_ROLE);
-    client.addRoleToGroups(ADMIN_USER, ADMIN_ROLE, Sets.newHashSet(ADMIN_GROUP));
-    client.grantCollectionPrivilege(SearchConstants.ALL, ADMIN_USER, ADMIN_ROLE, SearchConstants.ALL);
+    client.createRole(ADMIN_USER, ADMIN_ROLE, COMPONENT_SOLR);
+    client.addRoleToGroups(ADMIN_USER, ADMIN_ROLE, COMPONENT_SOLR, Sets.newHashSet(ADMIN_GROUP));
+    grantCollectionPrivilege(SearchConstants.ALL, ADMIN_USER, ADMIN_ROLE, SearchConstants.ALL);
+  }
+
+  protected static void grantCollectionPrivilege(String collection, String requestor,
+      String roleName, String action) throws SentryUserException {
+    TSentryPrivilege tPrivilege = toTSentryPrivilege(collection, action);
+    client.grantPrivilege(requestor, roleName, COMPONENT_SOLR, tPrivilege);
+  }
+
+  protected static void revokeCollectionPrivilege(String collection, String requestor,
+      String roleName, String action) throws SentryUserException {
+    TSentryPrivilege tPrivilege = toTSentryPrivilege(collection, action);
+    client.revokePrivilege(requestor, roleName, COMPONENT_SOLR, tPrivilege);
+  }
+
+  protected static void dropCollectionPrivilege(String collection, String requestor)
+      throws SentryUserException {
+    final TSentryPrivilege tPrivilege = toTSentryPrivilege(collection, Action.ALL);
+    client.dropPrivilege(requestor, COMPONENT_SOLR, tPrivilege);
+  }
+
+  private static TSentryPrivilege toTSentryPrivilege(String collection, String action) {
+    TSentryPrivilege tPrivilege = new TSentryPrivilege();
+    tPrivilege.setComponent(COMPONENT_SOLR);
+    tPrivilege.setServiceName(CLUSTER_NAME);
+    tPrivilege.setAction(action);
+    tPrivilege.setGrantOption(TSentryGrantOption.FALSE);
+
+    List<TAuthorizable> authorizables = Lists.newArrayList(new TAuthorizable(Collection.name(),
+        collection));
+    tPrivilege.setAuthorizables(authorizables);
+    return tPrivilege;
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/35c62ffc/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/TestSolrAdminOperations.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/TestSolrAdminOperations.java b/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/TestSolrAdminOperations.java
index 00a7a89..69b9066 100644
--- a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/TestSolrAdminOperations.java
+++ b/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/TestSolrAdminOperations.java
@@ -17,6 +17,8 @@
 package org.apache.sentry.tests.e2e.solr.db.integration;
 
 
+import static org.junit.Assert.assertTrue;
+
 import java.io.File;
 import java.util.Arrays;
 
@@ -27,8 +29,6 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.junit.Assert.assertTrue;
-
 public class TestSolrAdminOperations extends AbstractSolrSentryTestWithDbProvider {
   private static final Logger LOG = LoggerFactory.getLogger(TestSolrAdminOperations.class);
   private static final String TEST_COLLECTION_NAME1 = "collection1";
@@ -52,8 +52,8 @@ public class TestSolrAdminOperations extends AbstractSolrSentryTestWithDbProvide
      * user0->group0->role0
      * grant ALL privilege on collection admin and collection1 to role0
      */
-    client.grantCollectionPrivilege(ADMIN_COLLECTION_NAME, ADMIN_USER, "role0", SearchConstants.ALL);
-    client.grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role0", SearchConstants.ALL);
+    grantCollectionPrivilege(ADMIN_COLLECTION_NAME, ADMIN_USER, "role0", SearchConstants.ALL);
+    grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role0", SearchConstants.ALL);
 
     verifyCollectionAdminOpPass(grantor, CollectionAction.CREATE, TEST_COLLECTION_NAME1);
     verifyCollectionAdminOpPass(grantor, CollectionAction.RELOAD, TEST_COLLECTION_NAME1);
@@ -62,7 +62,7 @@ public class TestSolrAdminOperations extends AbstractSolrSentryTestWithDbProvide
     verifyCollectionAdminOpPass(grantor, CollectionAction.DELETE, TEST_COLLECTION_NAME1);
 
     //revoke UPDATE privilege on collection collection1 from role1, create collection1 will be failed
-    client.revokeCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role0", SearchConstants.UPDATE);
+    revokeCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role0", SearchConstants.UPDATE);
 
     verifyCollectionAdminOpFail(grantor, CollectionAction.CREATE, TEST_COLLECTION_NAME1);
     verifyCollectionAdminOpFail(grantor, CollectionAction.RELOAD, TEST_COLLECTION_NAME1);
@@ -75,8 +75,8 @@ public class TestSolrAdminOperations extends AbstractSolrSentryTestWithDbProvide
      * grant UPDATE privilege on collection admin and collection1 to role1
      */
     grantor = "user1";
-    client.grantCollectionPrivilege(ADMIN_COLLECTION_NAME, ADMIN_USER, "role1", SearchConstants.UPDATE);
-    client.grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role1", SearchConstants.UPDATE);
+    grantCollectionPrivilege(ADMIN_COLLECTION_NAME, ADMIN_USER, "role1", SearchConstants.UPDATE);
+    grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role1", SearchConstants.UPDATE);
 
     verifyCollectionAdminOpPass(grantor, CollectionAction.CREATE, TEST_COLLECTION_NAME1);
     verifyCollectionAdminOpPass(grantor, CollectionAction.RELOAD, TEST_COLLECTION_NAME1);
@@ -85,7 +85,7 @@ public class TestSolrAdminOperations extends AbstractSolrSentryTestWithDbProvide
     verifyCollectionAdminOpPass(grantor, CollectionAction.DELETE, TEST_COLLECTION_NAME1);
 
     //revoke UPDATE privilege on collection admin from role1, create collection1 will be failed
-    client.revokeCollectionPrivilege(ADMIN_COLLECTION_NAME, ADMIN_USER, "role1", SearchConstants.UPDATE);
+    revokeCollectionPrivilege(ADMIN_COLLECTION_NAME, ADMIN_USER, "role1", SearchConstants.UPDATE);
     verifyCollectionAdminOpFail(grantor, CollectionAction.CREATE, TEST_COLLECTION_NAME1);
     verifyCollectionAdminOpFail(grantor, CollectionAction.RELOAD, TEST_COLLECTION_NAME1);
     verifyCollectionAdminOpFail(grantor, CollectionAction.CREATEALIAS, TEST_COLLECTION_NAME1);
@@ -98,8 +98,8 @@ public class TestSolrAdminOperations extends AbstractSolrSentryTestWithDbProvide
      * grant QUERY privilege on collection admin and collection1 to role2
      */
     grantor = "user2";
-    client.grantCollectionPrivilege(ADMIN_COLLECTION_NAME, ADMIN_USER, "role2", SearchConstants.QUERY);
-    client.grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role2", SearchConstants.QUERY);
+    grantCollectionPrivilege(ADMIN_COLLECTION_NAME, ADMIN_USER, "role2", SearchConstants.QUERY);
+    grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role2", SearchConstants.QUERY);
 
     verifyCollectionAdminOpFail(grantor, CollectionAction.CREATE, TEST_COLLECTION_NAME1);
     verifyCollectionAdminOpFail(grantor, CollectionAction.RELOAD, TEST_COLLECTION_NAME1);
@@ -108,11 +108,11 @@ public class TestSolrAdminOperations extends AbstractSolrSentryTestWithDbProvide
     verifyCollectionAdminOpFail(grantor, CollectionAction.DELETE, TEST_COLLECTION_NAME1);
 
     //grant UPDATE privilege on collection collection1 to role2, create collection1 will be failed
-    client.grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role2", SearchConstants.UPDATE);
+    grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role2", SearchConstants.UPDATE);
     verifyCollectionAdminOpFail(grantor, CollectionAction.CREATE, TEST_COLLECTION_NAME1);
 
     //grant UPDATE privilege on collection admin to role2, create collection1 will be successful.
-    client.grantCollectionPrivilege(ADMIN_COLLECTION_NAME, ADMIN_USER, "role2", SearchConstants.UPDATE);
+    grantCollectionPrivilege(ADMIN_COLLECTION_NAME, ADMIN_USER, "role2", SearchConstants.UPDATE);
 
     verifyCollectionAdminOpPass(grantor, CollectionAction.CREATE, TEST_COLLECTION_NAME1);
     verifyCollectionAdminOpPass(grantor, CollectionAction.RELOAD, TEST_COLLECTION_NAME1);
@@ -133,8 +133,8 @@ public class TestSolrAdminOperations extends AbstractSolrSentryTestWithDbProvide
      * grant UPDATE privilege on collection admin to role3
      * grant QUERY privilege on collection collection1 to role3
      */
-    client.grantCollectionPrivilege(ADMIN_COLLECTION_NAME, ADMIN_USER, "role3", SearchConstants.ALL);
-    client.grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role3", SearchConstants.ALL);
+    grantCollectionPrivilege(ADMIN_COLLECTION_NAME, ADMIN_USER, "role3", SearchConstants.ALL);
+    grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role3", SearchConstants.ALL);
 
     verifyCollectionAdminOpPass(grantor, CollectionAction.CREATE, TEST_COLLECTION_NAME1);
     verifyCollectionAdminOpPass(grantor, CollectionAction.RELOAD, TEST_COLLECTION_NAME1);
@@ -159,24 +159,27 @@ public class TestSolrAdminOperations extends AbstractSolrSentryTestWithDbProvide
      * Grant ALL privilege on collection admin to role0
      * user0 can execute create & delete collection1 operation
      */
-    client.grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role0", SearchConstants.ALL);
-    client.grantCollectionPrivilege(ADMIN_COLLECTION_NAME, ADMIN_USER, "role0", SearchConstants.ALL);
+    grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role0", SearchConstants.ALL);
+    grantCollectionPrivilege(ADMIN_COLLECTION_NAME, ADMIN_USER, "role0", SearchConstants.ALL);
 
     assertTrue("user0 has one privilege on collection admin",
-        client.listPrivilegesByRoleName("user0", "role0", Arrays.asList(new Collection(ADMIN_COLLECTION_NAME))).size() == 1);
+        client.listPrivilegesByRoleName("user0", "role0", COMPONENT_SOLR, CLUSTER_NAME,
+            Arrays.asList(new Collection(ADMIN_COLLECTION_NAME))).size() == 1);
 
     assertTrue("user0 has one privilege on collection collection1",
-        client.listPrivilegesByRoleName("user0", "role0", Arrays.asList(new Collection(TEST_COLLECTION_NAME1))).size() == 1);
+        client.listPrivilegesByRoleName("user0", "role0", COMPONENT_SOLR, CLUSTER_NAME,
+            Arrays.asList(new Collection(TEST_COLLECTION_NAME1))).size() == 1);
 
     /**
      * user1->group1->role1
      * grant QUERY privilege on collection collection1 to role1
      */
 
-    client.listPrivilegesByRoleName("user0", "role0", null);
-    client.grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role1", SearchConstants.ALL);
+    client.listPrivilegesByRoleName("user0", "role0", COMPONENT_SOLR, CLUSTER_NAME, null);
+    grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role1", SearchConstants.ALL);
     assertTrue("user1 has one privilege record",
-        client.listPrivilegesByRoleName("user1", "role1", Arrays.asList(new Collection(TEST_COLLECTION_NAME1))).size() == 1);
+        client.listPrivilegesByRoleName("user1", "role1", COMPONENT_SOLR, CLUSTER_NAME,
+            Arrays.asList(new Collection(TEST_COLLECTION_NAME1))).size() == 1);
 
     /**
      * create collection collection1
@@ -189,32 +192,36 @@ public class TestSolrAdminOperations extends AbstractSolrSentryTestWithDbProvide
 
     //check the user0
     assertTrue("user0 has one privilege on collection admin",
-        client.listPrivilegesByRoleName("user0", "role0", Arrays.asList(new Collection(ADMIN_COLLECTION_NAME))).size() == 1);
+        client.listPrivilegesByRoleName("user0", "role0", COMPONENT_SOLR, CLUSTER_NAME,
+            Arrays.asList(new Collection(ADMIN_COLLECTION_NAME))).size() == 1);
 
     assertTrue("user0 has no privilege on collection collection1",
-        client.listPrivilegesByRoleName("user0", "role0", Arrays.asList(new Collection(TEST_COLLECTION_NAME1))).size() == 0);
+        client.listPrivilegesByRoleName("user0", "role0", COMPONENT_SOLR, CLUSTER_NAME,
+            Arrays.asList(new Collection(TEST_COLLECTION_NAME1))).size() == 0);
 
     //check the user1
     assertTrue("user1 has no privilege on collection collection1",
-        client.listPrivilegesByRoleName("user1", "role1", Arrays.asList(new Collection(TEST_COLLECTION_NAME1))).size() == 0);
-
+        client.listPrivilegesByRoleName("user1", "role1", COMPONENT_SOLR, CLUSTER_NAME,
+            Arrays.asList(new Collection(TEST_COLLECTION_NAME1))).size() == 0);
 
     /**
      * user2->group2->role2
      * Grant UPDATE privilege on collection collection1 to role2
      */
-    client.grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role2", SearchConstants.UPDATE);
+    grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role2", SearchConstants.UPDATE);
 
     assertTrue("user2 has one privilege on collection collection1",
-        client.listPrivilegesByRoleName("user2", "role2", Arrays.asList(new Collection(TEST_COLLECTION_NAME1))).size() == 1);
+        client.listPrivilegesByRoleName("user2", "role2", COMPONENT_SOLR, CLUSTER_NAME,
+            Arrays.asList(new Collection(TEST_COLLECTION_NAME1))).size() == 1);
 
     /**
      * user3->group3->role3
      * grant QUERY privilege on collection collection1 to role3
      */
-    client.grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role3", SearchConstants.QUERY);
+    grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role3", SearchConstants.QUERY);
     assertTrue("user1 has one privilege record",
-        client.listPrivilegesByRoleName("user3", "role3", Arrays.asList(new Collection(TEST_COLLECTION_NAME1))).size() == 1);
+        client.listPrivilegesByRoleName("user3", "role3", COMPONENT_SOLR, CLUSTER_NAME,
+            Arrays.asList(new Collection(TEST_COLLECTION_NAME1))).size() == 1);
 
     /**
      * create collection collection1
@@ -227,10 +234,12 @@ public class TestSolrAdminOperations extends AbstractSolrSentryTestWithDbProvide
 
     //check the user2
     assertTrue("user2 has no privilege on collection collection1",
-        client.listPrivilegesByRoleName("user2", "role2", Arrays.asList(new Collection(TEST_COLLECTION_NAME1))).size() == 0);
+        client.listPrivilegesByRoleName("user2", "role2", COMPONENT_SOLR, CLUSTER_NAME,
+            Arrays.asList(new Collection(TEST_COLLECTION_NAME1))).size() == 0);
 
     //check the user3
     assertTrue("user3 has no privilege on collection collection1",
-        client.listPrivilegesByRoleName("user3", "role3", Arrays.asList(new Collection(TEST_COLLECTION_NAME1))).size() == 0);
+        client.listPrivilegesByRoleName("user3", "role3", COMPONENT_SOLR, CLUSTER_NAME,
+            Arrays.asList(new Collection(TEST_COLLECTION_NAME1))).size() == 0);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/35c62ffc/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/TestSolrDocLevelOperations.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/TestSolrDocLevelOperations.java b/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/TestSolrDocLevelOperations.java
index 193743b..7f1fdfd 100644
--- a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/TestSolrDocLevelOperations.java
+++ b/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/TestSolrDocLevelOperations.java
@@ -64,14 +64,14 @@ public class TestSolrDocLevelOperations extends AbstractSolrSentryTestWithDbProv
 
       // as user0
       setAuthenticationUser("user0");
-      client.grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role0", SearchConstants.QUERY);
+      grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role0", SearchConstants.QUERY);
       rsp = server.query(query);
       docList = rsp.getResults();
       assertEquals(NUM_DOCS/4, rsp.getResults().getNumFound());
 
       //as user1
       setAuthenticationUser("user1");
-      client.grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role1", SearchConstants.QUERY);
+      grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role1", SearchConstants.QUERY);
       rsp = server.query(query);
       docList = rsp.getResults();
       assertEquals(NUM_DOCS/4, rsp.getResults().getNumFound());  docList = rsp.getResults();
@@ -79,14 +79,14 @@ public class TestSolrDocLevelOperations extends AbstractSolrSentryTestWithDbProv
 
       //as user2
       setAuthenticationUser("user2");
-      client.grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role2", SearchConstants.QUERY);
+      grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role2", SearchConstants.QUERY);
       rsp = server.query(query);
       docList = rsp.getResults();
       assertEquals(NUM_DOCS/4, rsp.getResults().getNumFound());
 
       //as user3
       setAuthenticationUser("user3");
-      client.grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role3", SearchConstants.QUERY);
+      grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role3", SearchConstants.QUERY);
       rsp = server.query(query);
       docList = rsp.getResults();
       assertEquals(NUM_DOCS/4, rsp.getResults().getNumFound());
@@ -106,7 +106,7 @@ public class TestSolrDocLevelOperations extends AbstractSolrSentryTestWithDbProv
     CloudSolrServer server = getCloudSolrServer(TEST_COLLECTION_NAME1);
     try {
       setAuthenticationUser("user0");
-      client.grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role0", SearchConstants.QUERY);
+      grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role0", SearchConstants.QUERY);
       String docIdStr = Long.toString(1);
 
       // verify we can't view one of the odd documents

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/35c62ffc/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/TestSolrQueryOperations.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/TestSolrQueryOperations.java b/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/TestSolrQueryOperations.java
index afe6912..663350d 100644
--- a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/TestSolrQueryOperations.java
+++ b/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/TestSolrQueryOperations.java
@@ -18,13 +18,14 @@ package org.apache.sentry.tests.e2e.solr.db.integration;
 
 import java.io.File;
 
+import org.apache.sentry.core.model.search.Collection;
 import org.apache.sentry.core.model.search.SearchConstants;
 import org.apache.solr.common.SolrInputDocument;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.Sets;
+import com.google.common.collect.Lists;
 
 public class TestSolrQueryOperations extends AbstractSolrSentryTestWithDbProvider {
   private static final Logger LOG = LoggerFactory.getLogger(TestSolrQueryOperations.class);
@@ -54,13 +55,13 @@ public class TestSolrQueryOperations extends AbstractSolrSentryTestWithDbProvide
      * grant ALL privilege on collection collection1 to role0
      */
     String grantor = "user0";
-    client.grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role0", SearchConstants.ALL);
+    grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role0", SearchConstants.ALL);
     verifyQueryPass(grantor, TEST_COLLECTION_NAME1, ALL_DOCS);
 
-    client.revokeCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role0", SearchConstants.UPDATE);
+    revokeCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role0", SearchConstants.UPDATE);
     verifyQueryPass(grantor, TEST_COLLECTION_NAME1, ALL_DOCS);
 
-    client.revokeCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role0", SearchConstants.QUERY);
+    revokeCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role0", SearchConstants.QUERY);
     verifyQueryFail(grantor, TEST_COLLECTION_NAME1, ALL_DOCS);
 
     /**
@@ -68,10 +69,10 @@ public class TestSolrQueryOperations extends AbstractSolrSentryTestWithDbProvide
      * grant QUERY privilege on collection collection1 to role1
      */
     grantor = "user1";
-    client.grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role1", SearchConstants.QUERY);
+    grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role1", SearchConstants.QUERY);
     verifyQueryPass(grantor, TEST_COLLECTION_NAME1, ALL_DOCS);
 
-    client.revokeCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role1", SearchConstants.QUERY);
+    revokeCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role1", SearchConstants.QUERY);
     verifyQueryFail(grantor, TEST_COLLECTION_NAME1, ALL_DOCS);
 
     /**
@@ -79,13 +80,15 @@ public class TestSolrQueryOperations extends AbstractSolrSentryTestWithDbProvide
      * grant UPDATE privilege on collection collection1 to role2
      */
     grantor = "user2";
-    client.grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role2", SearchConstants.UPDATE);
+    grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role2", SearchConstants.UPDATE);
     verifyQueryFail(grantor, TEST_COLLECTION_NAME1, ALL_DOCS);
 
-    client.grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role2", SearchConstants.QUERY);
+    grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role2", SearchConstants.QUERY);
     verifyQueryPass(grantor, TEST_COLLECTION_NAME1, ALL_DOCS);
 
-    client.renameCollectionPrivilege(TEST_COLLECTION_NAME1, "new_" + TEST_COLLECTION_NAME1, ADMIN_USER);
+    client.renamePrivilege(ADMIN_USER, COMPONENT_SOLR, CLUSTER_NAME,
+        Lists.newArrayList(new Collection(TEST_COLLECTION_NAME1)),
+        Lists.newArrayList(new Collection("new_" + TEST_COLLECTION_NAME1)));
     verifyQueryFail(grantor, TEST_COLLECTION_NAME1, ALL_DOCS);
 
     grantor = "user3";

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/35c62ffc/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/TestSolrUpdateOperations.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/TestSolrUpdateOperations.java b/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/TestSolrUpdateOperations.java
index de18979..765fc34 100644
--- a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/TestSolrUpdateOperations.java
+++ b/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/db/integration/TestSolrUpdateOperations.java
@@ -24,8 +24,6 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.Sets;
-
 public class TestSolrUpdateOperations extends AbstractSolrSentryTestWithDbProvider {
   private static final Logger LOG = LoggerFactory.getLogger(TestSolrUpdateOperations.class);
   private static final String TEST_COLLECTION_NAME1 = "collection1";
@@ -51,13 +49,13 @@ public class TestSolrUpdateOperations extends AbstractSolrSentryTestWithDbProvid
      * grant ALL privilege on collection collection1 to role0
      */
     String grantor = "user0";
-    client.grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role0", SearchConstants.ALL);
+    grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role0", SearchConstants.ALL);
     cleanSolrCollection(TEST_COLLECTION_NAME1);
     verifyUpdatePass(grantor, TEST_COLLECTION_NAME1, solrInputDoc);
     verifyDeletedocsPass(grantor, TEST_COLLECTION_NAME1, false);
 
     //drop privilege
-    client.dropCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER);
+    dropCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER);
     verifyUpdateFail(grantor, TEST_COLLECTION_NAME1, solrInputDoc);
     uploadSolrDoc(TEST_COLLECTION_NAME1, solrInputDoc);
     verifyDeletedocsFail(grantor, TEST_COLLECTION_NAME1, false);
@@ -67,13 +65,13 @@ public class TestSolrUpdateOperations extends AbstractSolrSentryTestWithDbProvid
      * grant UPDATE privilege on collection collection1 to role1
      */
     grantor = "user1";
-    client.grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role1", SearchConstants.UPDATE);
+    grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role1", SearchConstants.UPDATE);
     cleanSolrCollection(TEST_COLLECTION_NAME1);
     verifyUpdatePass(grantor, TEST_COLLECTION_NAME1, solrInputDoc);
     verifyDeletedocsPass(grantor, TEST_COLLECTION_NAME1, false);
 
     //revoke privilege
-    client.revokeCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role1", SearchConstants.ALL);
+    revokeCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role1", SearchConstants.ALL);
     verifyUpdateFail(grantor, TEST_COLLECTION_NAME1, solrInputDoc);
     uploadSolrDoc(TEST_COLLECTION_NAME1, solrInputDoc);
     verifyDeletedocsFail(grantor, TEST_COLLECTION_NAME1, false);
@@ -83,13 +81,13 @@ public class TestSolrUpdateOperations extends AbstractSolrSentryTestWithDbProvid
      * grant QUERY privilege on collection collection1 to role2
      */
     grantor = "user2";
-    client.grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role2", SearchConstants.QUERY);
+    grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role2", SearchConstants.QUERY);
     cleanSolrCollection(TEST_COLLECTION_NAME1);
     verifyUpdateFail(grantor, TEST_COLLECTION_NAME1, solrInputDoc);
     uploadSolrDoc(TEST_COLLECTION_NAME1, solrInputDoc);
     verifyDeletedocsFail(grantor, TEST_COLLECTION_NAME1, false);
 
-    client.grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role2", SearchConstants.ALL);
+    grantCollectionPrivilege(TEST_COLLECTION_NAME1, ADMIN_USER, "role2", SearchConstants.ALL);
     cleanSolrCollection(TEST_COLLECTION_NAME1);
     verifyUpdatePass(grantor, TEST_COLLECTION_NAME1, solrInputDoc);
     verifyDeletedocsPass(grantor, TEST_COLLECTION_NAME1, false);

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/35c62ffc/sentry-tests/sentry-tests-sqoop/src/test/java/org/apache/sentry/tests/e2e/sqoop/AbstractSqoopSentryTestBase.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-sqoop/src/test/java/org/apache/sentry/tests/e2e/sqoop/AbstractSqoopSentryTestBase.java b/sentry-tests/sentry-tests-sqoop/src/test/java/org/apache/sentry/tests/e2e/sqoop/AbstractSqoopSentryTestBase.java
index bb8ceb5..93ccd75 100644
--- a/sentry-tests/sentry-tests-sqoop/src/test/java/org/apache/sentry/tests/e2e/sqoop/AbstractSqoopSentryTestBase.java
+++ b/sentry-tests/sentry-tests-sqoop/src/test/java/org/apache/sentry/tests/e2e/sqoop/AbstractSqoopSentryTestBase.java
@@ -30,13 +30,12 @@ import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.TimeoutException;
 
-
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
-
 import org.apache.sentry.core.model.sqoop.SqoopActionConstant;
+import org.apache.sentry.provider.db.generic.SentryGenericProviderBackend;
 import org.apache.sentry.provider.db.generic.service.thrift.SentryGenericServiceClient;
 import org.apache.sentry.provider.db.generic.service.thrift.TAuthorizable;
 import org.apache.sentry.provider.db.generic.service.thrift.TSentryPrivilege;
@@ -46,22 +45,14 @@ import org.apache.sentry.service.thrift.SentryService;
 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.sqoop.binding.SqoopProviderBackend;
 import org.apache.sentry.sqoop.conf.SqoopAuthConf.AuthzConfVars;
-
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Joiner;
 import com.google.common.collect.Sets;
 
 public class AbstractSqoopSentryTestBase {
-  private static final Logger LOGGER = LoggerFactory
-      .getLogger(AbstractSqoopSentryTestBase.class);
-
   private static final String SERVER_HOST = NetUtils
       .createSocketAddr("localhost:80").getAddress().getCanonicalHostName();
   private static final int PORT = 8038;
@@ -185,7 +176,8 @@ public class AbstractSqoopSentryTestBase {
 
     conf.set(AuthzConfVars.AUTHZ_PROVIDER.getVar(),
         LocalGroupResourceAuthorizationProvider.class.getName());
-    conf.set(AuthzConfVars.AUTHZ_PROVIDER_BACKEND.getVar(), SqoopProviderBackend.class.getName());
+    conf.set(AuthzConfVars.AUTHZ_PROVIDER_BACKEND.getVar(),
+        SentryGenericProviderBackend.class.getName());
     conf.set(AuthzConfVars.AUTHZ_PROVIDER_RESOURCE.getVar(), policyFilePath.getPath());
     conf.set(AuthzConfVars.AUTHZ_TESTING_MODE.getVar(), "true");
     return conf;