You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by li...@apache.org on 2016/10/04 16:31:38 UTC

sentry git commit: SENTRY-1478: Disable fencing in Sentry store for Active/Active (Li Li, Reviewed by: Hao Hao)

Repository: sentry
Updated Branches:
  refs/heads/sentry-ha-redesign 92ebf1d46 -> f048132a7


SENTRY-1478: Disable fencing in Sentry store for Active/Active  (Li Li, Reviewed by: Hao Hao)

Change-Id: Idfd6e3f783fd0b4d81fdee109306be0d74df5440


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

Branch: refs/heads/sentry-ha-redesign
Commit: f048132a7e7e111ea0d2f28220cc992b66ed95f8
Parents: 92ebf1d
Author: lili <li...@cloudera.com>
Authored: Thu Sep 22 17:01:30 2016 -0700
Committer: lili <li...@cloudera.com>
Committed: Tue Oct 4 09:16:36 2016 -0700

----------------------------------------------------------------------
 .../service/persistent/DelegateSentryStore.java | 24 ++------
 .../service/persistent/SentryStoreLayer.java    |  9 ++-
 .../db/service/persistent/SentryStore.java      | 62 +++++--------------
 .../apache/sentry/service/thrift/Activator.java |  4 +-
 .../sentry/service/thrift/HMSFollower.java      | 32 ++--------
 .../thrift/PoolClientInvocationHandler.java     | 34 +++--------
 .../sentry/service/thrift/SentryService.java    | 19 +-----
 .../persistent/SentryStoreIntegrationBase.java  | 14 -----
 .../TestPrivilegeOperatePersistence.java        | 19 +-----
 .../db/service/persistent/TestFencer.java       | 64 --------------------
 .../db/service/persistent/TestSentryStore.java  | 16 +----
 .../persistent/TestSentryStoreImportExport.java | 16 +----
 .../service/persistent/TestSentryVersion.java   | 17 ------
 .../sentry/tests/e2e/ha/TestFailover.java       | 34 +----------
 14 files changed, 48 insertions(+), 316 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sentry/blob/f048132a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/persistent/DelegateSentryStore.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/persistent/DelegateSentryStore.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/persistent/DelegateSentryStore.java
index 6f8239d..bac479f 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/persistent/DelegateSentryStore.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/persistent/DelegateSentryStore.java
@@ -28,7 +28,6 @@ import javax.jdo.PersistenceManager;
 import javax.jdo.Query;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.sentry.core.common.exception.SentryStandbyException;
 import org.apache.sentry.core.common.exception.SentryUserException;
 import org.apache.sentry.core.common.Authorizable;
 import org.apache.sentry.core.common.exception.SentryAccessDeniedException;
@@ -45,8 +44,6 @@ import org.apache.sentry.provider.db.service.persistent.SentryStore;
 import org.apache.sentry.provider.db.service.thrift.SentryPolicyStoreProcessor;
 import org.apache.sentry.provider.db.service.thrift.TSentryGroup;
 import org.apache.sentry.provider.db.service.thrift.TSentryRole;
-import org.apache.sentry.service.thrift.Activator;
-import org.apache.sentry.service.thrift.Activators;
 import org.apache.sentry.service.thrift.ServiceConstants.ServerConfig;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -68,12 +65,9 @@ public class DelegateSentryStore implements SentryStoreLayer {
   private final Configuration conf;
   private final Set<String> adminGroups;
   private final PrivilegeOperatePersistence privilegeOperator;
-  private final Activator act;
 
   public DelegateSentryStore(Configuration conf) throws SentryNoSuchObjectException,
-      SentryAccessDeniedException, SentrySiteConfigurationException, IOException,
-          SentryStandbyException {
-    this.act = Activators.INSTANCE.get(conf);
+      SentryAccessDeniedException, SentrySiteConfigurationException, IOException {
     this.privilegeOperator = new PrivilegeOperatePersistence(conf);
     // The generic model doesn't turn on the thread that cleans hive privileges
     conf.set(ServerConfig.SENTRY_STORE_ORPHANED_PRIVILEGE_REMOVAL,"false");
@@ -106,8 +100,7 @@ public class DelegateSentryStore implements SentryStoreLayer {
 
   @Override
   public CommitContext createRole(String component, String role,
-      String requestor) throws SentryAlreadyExistsException,
-          SentryStandbyException {
+      String requestor) throws SentryAlreadyExistsException {
     return delegate.createSentryRole(role);
   }
 
@@ -117,13 +110,12 @@ public class DelegateSentryStore implements SentryStoreLayer {
    */
   @Override
   public CommitContext dropRole(String component, String role, String requestor)
-      throws SentryNoSuchObjectException, SentryStandbyException {
+      throws SentryNoSuchObjectException {
     boolean rollbackTransaction = true;
     PersistenceManager pm = null;
     String trimmedRole = toTrimmedLower(role);
     try {
       pm = openTransaction();
-      act.checkSqlFencing(pm);
       Query query = pm.newQuery(MSentryRole.class);
       query.setFilter("this.roleName == t");
       query.declareParameters("java.lang.String t");
@@ -154,15 +146,13 @@ public class DelegateSentryStore implements SentryStoreLayer {
 
   @Override
   public CommitContext alterRoleAddGroups(String component, String role,
-      Set<String> groups, String requestor) throws SentryNoSuchObjectException,
-          SentryStandbyException {
+      Set<String> groups, String requestor) throws SentryNoSuchObjectException {
     return delegate.alterSentryRoleAddGroups(requestor, role, toTSentryGroups(groups));
   }
 
   @Override
   public CommitContext alterRoleDeleteGroups(String component, String role,
-      Set<String> groups, String requestor) throws SentryNoSuchObjectException,
-          SentryStandbyException {
+      Set<String> groups, String requestor) throws SentryNoSuchObjectException {
   //called to old sentryStore
     return delegate.alterSentryRoleDeleteGroups(role, toTSentryGroups(groups));
   }
@@ -176,7 +166,6 @@ public class DelegateSentryStore implements SentryStoreLayer {
     boolean rollbackTransaction = true;
     try{
       pm = openTransaction();
-      act.checkSqlFencing(pm);
       MSentryRole mRole = getRole(trimmedRole, pm);
       if (mRole == null) {
         throw new SentryNoSuchObjectException("Role: " + trimmedRole + " doesn't exist");
@@ -208,7 +197,6 @@ public class DelegateSentryStore implements SentryStoreLayer {
     boolean rollbackTransaction = true;
     try{
       pm = openTransaction();
-      act.checkSqlFencing(pm);
       MSentryRole mRole = getRole(trimmedRole, pm);
       if (mRole == null) {
         throw new SentryNoSuchObjectException("Role: " + trimmedRole + " doesn't exist");
@@ -252,7 +240,6 @@ public class DelegateSentryStore implements SentryStoreLayer {
     boolean rollbackTransaction = true;
     try {
       pm = openTransaction();
-      act.checkSqlFencing(pm);
 
       privilegeOperator.renamePrivilege(toTrimmedLower(component), toTrimmedLower(service),
           oldAuthorizables, newAuthorizables, requestor, pm);
@@ -276,7 +263,6 @@ public class DelegateSentryStore implements SentryStoreLayer {
     boolean rollbackTransaction = true;
     try {
       pm = openTransaction();
-      act.checkSqlFencing(pm);
 
       privilegeOperator.dropPrivilege(privilege, pm);
 

http://git-wip-us.apache.org/repos/asf/sentry/blob/f048132a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/persistent/SentryStoreLayer.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/persistent/SentryStoreLayer.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/persistent/SentryStoreLayer.java
index 39e55c3..c003965 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/persistent/SentryStoreLayer.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/persistent/SentryStoreLayer.java
@@ -20,7 +20,6 @@ package org.apache.sentry.provider.db.generic.service.persistent;
 import java.util.List;
 import java.util.Set;
 
-import org.apache.sentry.core.common.exception.SentryStandbyException;
 import org.apache.sentry.core.common.exception.SentryUserException;
 import org.apache.sentry.core.common.Authorizable;
 import org.apache.sentry.core.common.exception.SentryAlreadyExistsException;
@@ -41,7 +40,7 @@ public interface SentryStoreLayer {
    * @throws SentryAlreadyExistsException
    */
   CommitContext createRole(String component, String role,
-      String requestor) throws SentryAlreadyExistsException, SentryStandbyException;
+      String requestor) throws SentryAlreadyExistsException;
 
   /**
    * Drop a role
@@ -52,7 +51,7 @@ public interface SentryStoreLayer {
    * @throws SentryNoSuchObjectException
    */
   CommitContext dropRole(String component, String role,
-      String requestor) throws SentryNoSuchObjectException, SentryStandbyException;
+      String requestor) throws SentryNoSuchObjectException;
 
   /**
    * Add a role to groups.
@@ -64,7 +63,7 @@ public interface SentryStoreLayer {
    * @throws SentryNoSuchObjectException
    */
   CommitContext alterRoleAddGroups(String component, String role,
-      Set<String> groups, String requestor) throws SentryNoSuchObjectException, SentryStandbyException;
+      Set<String> groups, String requestor) throws SentryNoSuchObjectException;
 
   /**
    * Delete a role from groups.
@@ -76,7 +75,7 @@ public interface SentryStoreLayer {
    * @throws SentryNoSuchObjectException
    */
   CommitContext alterRoleDeleteGroups(String component, String role,
-      Set<String> groups, String requestor) throws SentryNoSuchObjectException, SentryStandbyException;
+      Set<String> groups, String requestor) throws SentryNoSuchObjectException;
 
   /**
    * Grant a privilege to role.

http://git-wip-us.apache.org/repos/asf/sentry/blob/f048132a/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 e2b92f9..26d7c5e 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
@@ -46,7 +46,6 @@ import javax.jdo.Transaction;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.sentry.core.common.exception.SentryStandbyException;
 import org.apache.sentry.core.common.exception.SentryUserException;
 import org.apache.sentry.core.common.utils.SentryConstants;
 import org.apache.sentry.core.common.exception.SentrySiteConfigurationException;
@@ -72,8 +71,6 @@ import org.apache.sentry.provider.db.service.thrift.TSentryMappingData;
 import org.apache.sentry.provider.db.service.thrift.TSentryPrivilege;
 import org.apache.sentry.provider.db.service.thrift.TSentryPrivilegeMap;
 import org.apache.sentry.provider.db.service.thrift.TSentryRole;
-import org.apache.sentry.service.thrift.Activator;
-import org.apache.sentry.service.thrift.Activators;
 import org.apache.sentry.service.thrift.ServiceConstants.PrivilegeScope;
 import org.apache.sentry.service.thrift.ServiceConstants.ServerConfig;
 import org.datanucleus.store.rdbms.exceptions.MissingTableException;
@@ -118,11 +115,6 @@ public class SentryStore {
       AccessConstants.ACTION_ALL.toLowerCase(), AccessConstants.SELECT, AccessConstants.INSERT);
 
   /**
-   * The activator object which tells us whether the current daemon is active.
-   */
-  private final Activator act;
-
-  /**
    * Commit order sequence id. This is used by notification handlers
    * to know the order in which events where committed to the database.
    * This instance variable is incremented in incrementGetSequenceId
@@ -177,9 +169,7 @@ public class SentryStore {
 
   public SentryStore(Configuration conf)
       throws SentryNoSuchObjectException, SentryAccessDeniedException,
-          SentrySiteConfigurationException, IOException,
-              SentryStandbyException {
-    this.act = Activators.INSTANCE.get(conf);
+          SentrySiteConfigurationException, IOException {
     commitSequenceId = 0;
     this.conf = conf;
     Properties prop = getDataNucleusProperties(conf);
@@ -207,8 +197,7 @@ public class SentryStore {
 
   // ensure that the backend DB schema is set
   public void verifySentryStoreSchema(boolean checkVersion)
-      throws SentryNoSuchObjectException, SentryAccessDeniedException,
-          SentryStandbyException {
+      throws SentryNoSuchObjectException, SentryAccessDeniedException {
     if (!checkVersion) {
       setSentryVersion(SentryStoreSchemaInfo.getSentryVersion(),
           "Schema version set implicitly");
@@ -323,12 +312,11 @@ public class SentryStore {
    * @throws SentryAlreadyExistsException
    */
   public CommitContext createSentryRole(String roleName)
-      throws SentryAlreadyExistsException, SentryStandbyException {
+      throws SentryAlreadyExistsException {
     boolean rollbackTransaction = true;
     PersistenceManager pm = null;
     try {
       pm = openTransaction();
-      act.checkSqlFencing(pm);
       createSentryRoleCore(pm, roleName);
       CommitContext commit = commitUpdateTransaction(pm);
       rollbackTransaction = false;
@@ -416,12 +404,11 @@ public class SentryStore {
   }
 
   @VisibleForTesting
-  void clearAllTables() throws SentryStandbyException {
+  void clearAllTables() {
     boolean rollbackTransaction = true;
     PersistenceManager pm = null;
     try {
       pm = openTransaction();
-      act.checkSqlFencing(pm);
       pm.newQuery(MSentryRole.class).deletePersistentAll();
       pm.newQuery(MSentryGroup.class).deletePersistentAll();
       pm.newQuery(MSentryUser.class).deletePersistentAll();
@@ -450,7 +437,6 @@ public class SentryStore {
     String trimmedRoleName = trimAndLower(roleName);
     try {
       pm = openTransaction();
-      act.checkSqlFencing(pm);
       for (TSentryPrivilege privilege : privileges) {
         // first do grant check
         grantOptionCheck(pm, grantorPrincipal, privilege);
@@ -542,7 +528,6 @@ public class SentryStore {
     String trimmedRoleName = safeTrimLower(roleName);
     try {
       pm = openTransaction();
-      act.checkSqlFencing(pm);
       for (TSentryPrivilege tPrivilege : tPrivileges) {
         // first do revoke check
         grantOptionCheck(pm, grantorPrincipal, tPrivilege);
@@ -807,12 +792,11 @@ public class SentryStore {
   }
 
   public CommitContext dropSentryRole(String roleName)
-      throws SentryNoSuchObjectException, SentryStandbyException {
+      throws SentryNoSuchObjectException {
     boolean rollbackTransaction = true;
     PersistenceManager pm = null;
     try {
       pm = openTransaction();
-      act.checkSqlFencing(pm);
       dropSentryRoleCore(pm, roleName);
       CommitContext commit = commitUpdateTransaction(pm);
       rollbackTransaction = false;
@@ -847,12 +831,11 @@ public class SentryStore {
 
   public CommitContext alterSentryRoleAddGroups(String grantorPrincipal, String roleName,
       Set<TSentryGroup> groupNames)
-      throws SentryNoSuchObjectException, SentryStandbyException {
+      throws SentryNoSuchObjectException {
     boolean rollbackTransaction = true;
     PersistenceManager pm = null;
     try {
       pm = openTransaction();
-      act.checkSqlFencing(pm);
       alterSentryRoleAddGroupsCore(pm, roleName, groupNames);
       CommitContext commit = commitUpdateTransaction(pm);
       rollbackTransaction = false;
@@ -894,13 +877,11 @@ public class SentryStore {
   }
 
   public CommitContext alterSentryRoleAddUsers(String roleName,
-      Set<String> userNames) throws SentryNoSuchObjectException,
-          SentryStandbyException {
+      Set<String> userNames) throws SentryNoSuchObjectException {
     boolean rollbackTransaction = true;
     PersistenceManager pm = null;
     try {
       pm = openTransaction();
-      act.checkSqlFencing(pm);
       alterSentryRoleAddUsersCore(pm, roleName, userNames);
       CommitContext commit = commitUpdateTransaction(pm);
       rollbackTransaction = false;
@@ -938,13 +919,12 @@ public class SentryStore {
   }
 
   public CommitContext alterSentryRoleDeleteUsers(String roleName, Set<String> userNames)
-      throws SentryNoSuchObjectException, SentryStandbyException {
+      throws SentryNoSuchObjectException {
     boolean rollbackTransaction = true;
     PersistenceManager pm = null;
     String trimmedRoleName = trimAndLower(roleName);
     try {
       pm = openTransaction();
-      act.checkSqlFencing(pm);
       MSentryRole role = getMSentryRole(pm, trimmedRoleName);
       if (role == null) {
         throw new SentryNoSuchObjectException("Role: " + trimmedRoleName);
@@ -976,13 +956,12 @@ public class SentryStore {
 
   public CommitContext alterSentryRoleDeleteGroups(String roleName,
       Set<TSentryGroup> groupNames)
-      throws SentryNoSuchObjectException, SentryStandbyException {
+      throws SentryNoSuchObjectException {
     boolean rollbackTransaction = true;
     PersistenceManager pm = null;
     String trimmedRoleName = trimAndLower(roleName);
     try {
       pm = openTransaction();
-      act.checkSqlFencing(pm);
       Query query = pm.newQuery(MSentryRole.class);
       query.setFilter("this.roleName == t");
       query.declareParameters("java.lang.String t");
@@ -1625,8 +1604,7 @@ public class SentryStore {
   }
 
   public void setSentryVersion(String newVersion, String verComment)
-      throws SentryNoSuchObjectException, SentryAccessDeniedException,
-          SentryStandbyException {
+      throws SentryNoSuchObjectException, SentryAccessDeniedException {
     MSentryVersion mVersion;
     boolean rollbackTransaction = true;
     PersistenceManager pm = null;
@@ -1645,7 +1623,6 @@ public class SentryStore {
     mVersion.setVersionComment(verComment);
     try {
       pm = openTransaction();
-      act.checkSqlFencing(pm);
       pm.makePersistent(mVersion);
       rollbackTransaction = false;
       commitTransaction(pm);
@@ -1695,15 +1672,13 @@ public class SentryStore {
    * Drop given privilege from all roles
    */
   public void dropPrivilege(TSentryAuthorizable tAuthorizable)
-      throws SentryNoSuchObjectException, SentryInvalidInputException,
-          SentryStandbyException {
+      throws SentryNoSuchObjectException, SentryInvalidInputException {
     PersistenceManager pm = null;
     boolean rollbackTransaction = true;
 
     TSentryPrivilege tPrivilege = toSentryPrivilege(tAuthorizable);
     try {
       pm = openTransaction();
-      act.checkSqlFencing(pm);
 
       if (isMultiActionsSupported(tPrivilege)) {
         for (String privilegeAction : ALL_ACTIONS) {
@@ -1734,8 +1709,7 @@ public class SentryStore {
    */
   public void renamePrivilege(TSentryAuthorizable tAuthorizable,
       TSentryAuthorizable newTAuthorizable)
-          throws SentryNoSuchObjectException, SentryInvalidInputException,
-              SentryStandbyException {
+          throws SentryNoSuchObjectException, SentryInvalidInputException {
     PersistenceManager pm = null;
     boolean rollbackTransaction = true;
 
@@ -1744,7 +1718,6 @@ public class SentryStore {
 
     try {
       pm = openTransaction();
-      act.checkSqlFencing(pm);
       // In case of tables or DBs, check all actions
       if (isMultiActionsSupported(tPrivilege)) {
         for (String privilegeAction : ALL_ACTIONS) {
@@ -2043,14 +2016,13 @@ public class SentryStore {
 
   public CommitContext createAuthzPathsMapping(String hiveObj,
       Set<String> paths) throws SentryNoSuchObjectException,
-          SentryAlreadyExistsException, SentryStandbyException {
+          SentryAlreadyExistsException {
 
     boolean rollbackTransaction = true;
     PersistenceManager pm = null;
 
     try {
       pm = openTransaction();
-      act.checkSqlFencing(pm);
       createAuthzPathsMappingCore(pm, hiveObj, paths);
       CommitContext commit = commitUpdateTransaction(pm);
       rollbackTransaction = false;
@@ -2146,9 +2118,7 @@ public class SentryStore {
           lock.unlock();
         }
         try {
-          if (act.isActive()) {
-            removeOrphanedPrivileges();
-          }
+          removeOrphanedPrivileges();
         } catch (Exception e) {
           LOGGER.warn("Privilege cleaning thread encountered an error: " +
                   e.getMessage());
@@ -2211,7 +2181,7 @@ public class SentryStore {
      * second transaction will go and get each of those privilege objects,
      * verify that there are no roles attached, and then delete them.
      */
-    private void removeOrphanedPrivileges() throws SentryStandbyException {
+    private void removeOrphanedPrivileges() {
       final String privDB = "SENTRY_DB_PRIVILEGE";
       final String privId = "DB_PRIVILEGE_ID";
       final String mapDB = "SENTRY_ROLE_DB_PRIVILEGE_MAP";
@@ -2232,7 +2202,6 @@ public class SentryStore {
         Transaction transaction = pm.currentTransaction();
         transaction.begin();
         transaction.setRollbackOnly();  // Makes the tx read-only
-        act.checkSqlFencing(pm);
         Query query = pm.newQuery("javax.jdo.query.SQL", privFilter);
         query.setClass(MSentryPrivilege.class);
         List<MSentryPrivilege> results = (List<MSentryPrivilege>) query.execute();
@@ -2615,7 +2584,6 @@ public class SentryStore {
     TSentryMappingData mappingData = lowercaseRoleName(tSentryMappingData);
     try {
       pm = openTransaction();
-      act.checkSqlFencing(pm);
       Set<String> existRoleNames = getAllRoleNames(pm);
       //
       Map<String, Set<TSentryGroup>> importedRoleGroupsMap = covertToRoleNameTGroupsMap(mappingData

http://git-wip-us.apache.org/repos/asf/sentry/blob/f048132a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/Activator.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/Activator.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/Activator.java
index a261d8d..c3df4d8 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/Activator.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/Activator.java
@@ -35,7 +35,9 @@ import javax.jdo.PersistenceManager;
 import javax.jdo.PersistenceManagerFactory;
 
 /**
- * The activator is used to access and modify the activation state of the sentry daemon.<p/>
+ * The activator is used to access and modify the activation state of the sentry daemon.
+ * In active / active mode, only one daemon can fetch snapshots from HMS and write to the
+ * the backend DB, thus we can use the Activator to mark it.
  */
 public class Activator implements Closeable {
   private static final Logger LOGGER = LoggerFactory.getLogger(Activator.class);

http://git-wip-us.apache.org/repos/asf/sentry/blob/f048132a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/HMSFollower.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/HMSFollower.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/HMSFollower.java
index 894fcc9..8989292 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/HMSFollower.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/HMSFollower.java
@@ -78,11 +78,7 @@ public class HMSFollower implements Runnable {
       SentryAccessDeniedException, SentrySiteConfigurationException, IOException { //TODO: Handle any possible exceptions or throw specific exceptions
     LOGGER.info("HMSFollower is being initialized");
     authzConf = conf;
-    try {
-      sentryStore = new SentryStore(authzConf);
-    } catch (SentryStandbyException e) {
-      //TODO: Do not start HMSFollower if standby
-    }
+    sentryStore = new SentryStore(authzConf);
     //TODO: Initialize currentEventID from Sentry db
     currentEventID = 0;
     this.fullUpdateComplete = fullUpdateComplete;
@@ -390,9 +386,6 @@ public class HMSFollower implements Runnable {
               LOGGER.info("Rename Sentry privilege ignored as there are no privileges on the table: %s.%s", oldDbName, oldTableName);
             } catch (SentryInvalidInputException e) {
               throw new SentryInvalidInputException("Could not process Alter table event. Event: " + event.toString(), e);
-            } catch (SentryStandbyException e) {
-              LOGGER.error("Seems like this process became a standby. " +
-                  "Ignoring the failure. The new leader will reprocess this notification event");
             }
           } else if(!oldLocation.equals(newLocation)) { // Only Location has changed{
             //- Alter table set location
@@ -417,38 +410,23 @@ public class HMSFollower implements Runnable {
   private void dropSentryDbPrivileges(String dbName) throws SentryNoSuchObjectException, SentryInvalidInputException {
     TSentryAuthorizable authorizable = new TSentryAuthorizable(hiveInstance);
     authorizable.setDb(dbName);
-    try {
-      sentryStore.dropPrivilege(authorizable);
-    } catch (SentryStandbyException e) {
-      LOGGER.error("Seems like this process became a standby. " +
-          "Ignoring the failure. The new leader will reprocess this notification event");
-    }
+    sentryStore.dropPrivilege(authorizable);
   }
   private void dropSentryTablePrivileges(String dbName, String tableName) throws SentryNoSuchObjectException,
       SentryInvalidInputException {
     TSentryAuthorizable authorizable = new TSentryAuthorizable(hiveInstance);
     authorizable.setDb(dbName);
     authorizable.setTable(tableName);
-    try {
-      sentryStore.dropPrivilege(authorizable);
-    } catch (SentryStandbyException e) {
-      LOGGER.error("Seems like this process became a standby. " +
-          "Ignoring the failure. The new leader will reprocess this notification event");
-    }
+    sentryStore.dropPrivilege(authorizable);
   }
   private void renamePrivileges(String oldDbName, String oldTableName, String newDbName, String newTableName) throws
-      SentryNoSuchObjectException, SentryInvalidInputException, SentryStandbyException{
+      SentryNoSuchObjectException, SentryInvalidInputException {
     TSentryAuthorizable oldAuthorizable = new TSentryAuthorizable(hiveInstance);
     oldAuthorizable.setDb(oldDbName);
     oldAuthorizable.setTable(oldTableName);
     TSentryAuthorizable newAuthorizable = new TSentryAuthorizable(hiveInstance);
     newAuthorizable.setDb(newDbName);
     newAuthorizable.setTable(newTableName);
-    try {
-      sentryStore.renamePrivilege(oldAuthorizable, newAuthorizable);
-    } catch (SentryStandbyException e) {
-      LOGGER.error("Seems like this process became a standby. " +
-          "Ignoring the failure. The new leader will reprocess this notification event");
-    }
+    sentryStore.renamePrivilege(oldAuthorizable, newAuthorizable);
   }
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/f048132a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/PoolClientInvocationHandler.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/PoolClientInvocationHandler.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/PoolClientInvocationHandler.java
index 353d461..842d5ca 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/PoolClientInvocationHandler.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/PoolClientInvocationHandler.java
@@ -26,7 +26,6 @@ import org.apache.commons.pool2.impl.AbandonedConfig;
 import org.apache.commons.pool2.impl.GenericObjectPool;
 import org.apache.commons.pool2.impl.GenericObjectPoolConfig;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.sentry.core.common.exception.SentryStandbyException;
 import org.apache.sentry.core.common.exception.SentryUserException;
 import org.apache.sentry.provider.db.service.thrift.SentryPolicyServiceClient;
 import org.apache.sentry.service.thrift.ServiceConstants.ClientConfig;
@@ -251,7 +250,7 @@ public class PoolClientInvocationHandler extends SentryClientInvocationHandler {
         // Try to make the RPC.
         ret = invokeFromPool(method, args, pool);
         break;
-      } catch (SentryStandbyException | TTransportException e) {
+      } catch (TTransportException e) {
         if (exc == null) {
           exc = new Exception[endpoints.length];
         }
@@ -271,30 +270,17 @@ public class PoolClientInvocationHandler extends SentryClientInvocationHandler {
       // Increase the retry num, and throw the exception if can't retry again.
       retryCount++;
       if (retryCount == connectionRetryTotal) {
-        boolean allStandby = true, allUnreachable = true;
         for (int i = 0; i < exc.length; i++) {
-          if (exc[i] instanceof SentryStandbyException) {
-            allUnreachable = false;
-            LOGGER.error("Sentry server " + endpoints[endpointIdx].getEndPointStr()
-                + " is in standby mode");
-          } else {
-            allStandby = false;
-            LOGGER.error("Sentry server " + endpoints[endpointIdx].getEndPointStr()
+          // Since freshestEndpointIdx is shared by multiple threads, it is possible that
+          // the ith endpoint has been tried in another thread and skipped in the current
+          // thread.
+          if (exc[i] != null) {
+            LOGGER.error("Sentry server " + endpoints[i].getEndPointStr()
                 + " is in unreachable.");
           }
         }
-        if (allStandby) {
-          throw new SentryStandbyException("All sentry servers are in " +
-              "standby mode.", lastExc);
-        } else if (allUnreachable) {
-          throw new SentryUserException("All sentry servers are unreachable. " +
-              "Diagnostics is needed for unreachable servers.",
-              lastExc);
-        } else {
-          throw new SentryUserException("All reachable servers are standby. " +
-              "Diagnostics is needed for unreachable servers.",
-              lastExc);
-        }
+        throw new SentryUserException("Sentry servers are unreachable. " +
+            "Diagnostics is needed for unreachable servers.", lastExc);
       }
     }
     return ret;
@@ -318,9 +304,7 @@ public class PoolClientInvocationHandler extends SentryClientInvocationHandler {
       // Get the target exception, check if SentryUserException or TTransportException is wrapped.
       // TTransportException means there has connection problem with the pool.
       Throwable targetException = e.getCause();
-      if (targetException instanceof SentryStandbyException) {
-        throw (SentryStandbyException)targetException;
-      } else if (targetException instanceof SentryUserException) {
+      if (targetException instanceof SentryUserException) {
         Throwable sentryTargetException = targetException.getCause();
         // If there has connection problem, eg, invalid connection if the service restarted,
         // sentryTargetException instanceof TTransportException = true.

http://git-wip-us.apache.org/repos/asf/sentry/blob/f048132a/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 7497719..9e9358b 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
@@ -335,13 +335,15 @@ public class SentryService implements Callable {
 
   /**
    * If the current daemon is active, make it standby.
+   * Here 'active' means it is the only daemon that can fetch snapshots from HMA and write
+   * to the backend DB.
    */
   @VisibleForTesting
   public synchronized void becomeStandby() throws Exception{
     try {
       if(act.isActive()) {
         LOGGER.info("Server with incarnation id: " + act.getIncarnationId() +
-                " becoming standby");
+            " becoming standby");
         act.deactivate();
       }
     } catch (Exception e) {
@@ -349,21 +351,6 @@ public class SentryService implements Callable {
     }
   }
 
-  /**
-   * If the current daemon is active, shutdown the server.
-   */
-  @VisibleForTesting
-  public synchronized void shutdownActive() throws Exception{
-    try {
-      if(act.isActive()) {
-        LOGGER.info("Stopping active server with incarnation id: " + act.getIncarnationId());
-        stop();
-      }
-    } catch (Exception e) {
-      LOGGER.error("Error while stopping the active sentry daemon", e);
-    }
-  }
-
   // wait for the service thread to finish execution
   public synchronized void waitOnFuture() throws ExecutionException, InterruptedException {
     LOGGER.info("Waiting on future.get()");

http://git-wip-us.apache.org/repos/asf/sentry/blob/f048132a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/service/persistent/SentryStoreIntegrationBase.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/service/persistent/SentryStoreIntegrationBase.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/service/persistent/SentryStoreIntegrationBase.java
index c52197f..f14b586 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/service/persistent/SentryStoreIntegrationBase.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/service/persistent/SentryStoreIntegrationBase.java
@@ -21,11 +21,7 @@ import java.io.File;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.sentry.provider.file.PolicyFile;
-import org.apache.sentry.service.thrift.Activator;
-import org.apache.sentry.service.thrift.Activators;
-import org.apache.sentry.service.thrift.ServiceConstants;
 import org.apache.sentry.service.thrift.ServiceConstants.ServerConfig;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -38,7 +34,6 @@ public abstract class SentryStoreIntegrationBase {
   private static File dataDir;
   private static File policyFilePath;
   protected static Configuration conf;
-  protected static Activator act;
   protected static DelegateSentryStore sentryStore;
   protected static PolicyFile policyFile;
 
@@ -62,8 +57,6 @@ public abstract class SentryStoreIntegrationBase {
     policyFilePath = new File(Files.createTempDir(), "local_policy_file.ini");
     conf.set(ServerConfig.SENTRY_STORE_GROUP_MAPPING_RESOURCE,
         policyFilePath.getPath());
-    act = Activators.INSTANCE.create(conf);
-		conf.set(ServiceConstants.CURRENT_INCARNATION_ID_KEY, act.getIncarnationId());
   }
 
   @After
@@ -73,9 +66,6 @@ public abstract class SentryStoreIntegrationBase {
 
   @AfterClass
   public static void teardown() {
-    if (act != null) {
-      IOUtils.cleanup(null, act);
-    }
     if (sentryStore != null) {
       sentryStore.close();
     }
@@ -85,10 +75,6 @@ public abstract class SentryStoreIntegrationBase {
     if (policyFilePath != null) {
       FileUtils.deleteQuietly(policyFilePath);
     }
-    if (act != null) {
-      Activators.INSTANCE.remove(act);
-      act = null;
-    }
   }
 
   public static void addGroupsToUser(String user, String... groupNames) {

http://git-wip-us.apache.org/repos/asf/sentry/blob/f048132a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/service/persistent/TestPrivilegeOperatePersistence.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/service/persistent/TestPrivilegeOperatePersistence.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/service/persistent/TestPrivilegeOperatePersistence.java
index fd10a7a..799d5ef 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/service/persistent/TestPrivilegeOperatePersistence.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/generic/service/persistent/TestPrivilegeOperatePersistence.java
@@ -37,8 +37,6 @@ import org.apache.sentry.core.model.sqoop.SqoopActionConstant;
 import org.apache.sentry.core.common.exception.SentryGrantDeniedException;
 import org.apache.sentry.provider.db.generic.service.persistent.PrivilegeObject.Builder;
 import org.apache.sentry.provider.file.PolicyFile;
-import org.apache.sentry.service.thrift.Activator;
-import org.apache.sentry.service.thrift.Activators;
 import org.apache.sentry.service.thrift.ServiceConstants;
 import org.junit.Before;
 import org.junit.Test;
@@ -989,13 +987,8 @@ public class TestPrivilegeOperatePersistence extends SentryStoreIntegrationBase
     Configuration confCopy = new Configuration(conf);
     confCopy.set(String.format(ServiceConstants.ServerConfig.SENTRY_COMPONENT_ACTION_FACTORY_FORMAT, externalComponent),
                  InvalidActionFactory.class.getName());
-    Activator act = Activators.INSTANCE.create(confCopy);
-    confCopy.set(ServiceConstants.CURRENT_INCARNATION_ID_KEY,
-                 act.getIncarnationId());
-   SentryStoreLayer store = new DelegateSentryStore(confCopy);
+    SentryStoreLayer store = new DelegateSentryStore(confCopy);
     testGrantPrivilege(store, externalComponent);
-    act.close();
-    Activators.INSTANCE.remove(act);
   }
 
   @Test
@@ -1004,13 +997,8 @@ public class TestPrivilegeOperatePersistence extends SentryStoreIntegrationBase
     Configuration confCopy = new Configuration(conf);
     confCopy.set(String.format(ServiceConstants.ServerConfig.SENTRY_COMPONENT_ACTION_FACTORY_FORMAT, externalComponent),
                  MyComponentActionFactory.class.getName());
-    Activator act = Activators.INSTANCE.create(confCopy);
-    confCopy.set(ServiceConstants.CURRENT_INCARNATION_ID_KEY,
-                 act.getIncarnationId());
     SentryStoreLayer store = new DelegateSentryStore(confCopy);
     testGrantPrivilege(store, externalComponent);
-    act.close();
-    Activators.INSTANCE.remove(act);
   }
 
   @Test
@@ -1019,13 +1007,8 @@ public class TestPrivilegeOperatePersistence extends SentryStoreIntegrationBase
     Configuration confCopy = new Configuration(conf);
     confCopy.set(String.format(ServiceConstants.ServerConfig.SENTRY_COMPONENT_ACTION_FACTORY_FORMAT, "mycomponent"),
                  MyComponentActionFactory.class.getName());
-    Activator act = Activators.INSTANCE.create(confCopy);
-    confCopy.set(ServiceConstants.CURRENT_INCARNATION_ID_KEY,
-                 act.getIncarnationId());
     SentryStoreLayer store = new DelegateSentryStore(confCopy);
     testGrantPrivilege(store, externalComponent);
-    act.close();
-    Activators.INSTANCE.remove(act);
   }
 
   private void testGrantPrivilege(SentryStoreLayer sentryStore, String component) throws SentryUserException {

http://git-wip-us.apache.org/repos/asf/sentry/blob/f048132a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestFencer.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestFencer.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestFencer.java
index 42b67ba..7c080fa 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestFencer.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestFencer.java
@@ -26,12 +26,7 @@ import java.util.Properties;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.sentry.core.common.exception.SentryStandbyException;
-import org.apache.sentry.service.thrift.Activator;
-import org.apache.sentry.service.thrift.Activators;
-import org.apache.sentry.service.thrift.ServiceConstants;
 import org.apache.sentry.service.thrift.ServiceConstants.ServerConfig;
-import org.junit.Assert;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -45,33 +40,6 @@ public class TestFencer {
   private static final Logger LOGGER = LoggerFactory
       .getLogger(TestFencer.class);
 
-  private static class ActivatorContext implements Closeable {
-    private final Configuration conf;
-    private final Activator act;
-
-    ActivatorContext(Configuration conf) throws Exception {
-      this.conf = new Configuration(conf);
-      this.act = Activators.INSTANCE.create(this.conf);
-      this.conf.set(ServiceConstants.CURRENT_INCARNATION_ID_KEY,
-          act.getIncarnationId());
-      this.conf.set(ServerConfig.SENTRY_VERIFY_SCHEM_VERSION, "false");
-    }
-
-    @Override
-    public void close() throws IOException {
-      this.act.close();
-      Activators.INSTANCE.remove(this.act);
-    }
-
-    public Configuration getConf() {
-      return conf;
-    }
-
-    public Activator getAct() {
-      return act;
-    }
-  }
-
   private static class DatabaseContext implements Closeable {
     private final Configuration conf;
     private final File dataDir;
@@ -116,36 +84,4 @@ public class TestFencer {
       }
     }
   }
-
-  @Test(timeout = 60000)
-  public void testDbModificationsInvokeFencer() throws Exception {
-    DatabaseContext dbCtx = new DatabaseContext();
-    Properties prop = SentryStore.getDataNucleusProperties(dbCtx.getConf());
-    PersistenceManagerFactory pmf = JDOHelper.
-        getPersistenceManagerFactory(prop);
-    ActivatorContext actCtx = new ActivatorContext(dbCtx.getConf());
-    Assert.assertTrue(actCtx.getAct().isActive());
-
-    // We should be able to modify the database version table.
-    SentryStore sentryStore = new SentryStore(actCtx.getConf());
-    sentryStore.setSentryVersion(SentryStoreSchemaInfo.getSentryVersion(),
-        "Schema version set by unit test");
-
-    // Unfencing the database should lead to SentryStandbyExceptions when we
-    // try to modify the version again.
-    actCtx.getAct().getFencer().unfence(pmf);
-    try {
-      sentryStore.setSentryVersion(
-          SentryStoreSchemaInfo.getSentryVersion() + "v2",
-          "Schema version set by unit test");
-      Assert.fail("Expected setSentryVersion to fail because we are " +
-          "unfenced.");
-    } catch (SentryStandbyException e) {
-    } finally {
-      sentryStore.stop();
-      IOUtils.cleanup(null, actCtx);
-      IOUtils.cleanup(null, dbCtx);
-      pmf.close();
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/f048132a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryStore.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryStore.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryStore.java
index 0634206..3ef1cf7 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryStore.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryStore.java
@@ -27,12 +27,10 @@ import java.util.Set;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.alias.CredentialProvider;
 import org.apache.hadoop.security.alias.CredentialProviderFactory;
 import org.apache.hadoop.security.alias.UserProvider;
 import org.apache.sentry.core.common.exception.SentryAccessDeniedException;
-import org.apache.sentry.core.common.exception.SentryStandbyException;
 import org.apache.sentry.core.model.db.AccessConstants;
 import org.apache.sentry.core.common.exception.SentryAlreadyExistsException;
 import org.apache.sentry.core.common.exception.SentryGrantDeniedException;
@@ -46,9 +44,6 @@ import org.apache.sentry.provider.db.service.thrift.TSentryGroup;
 import org.apache.sentry.provider.db.service.thrift.TSentryPrivilege;
 import org.apache.sentry.provider.db.service.thrift.TSentryRole;
 import org.apache.sentry.provider.file.PolicyFile;
-import org.apache.sentry.service.thrift.Activator;
-import org.apache.sentry.service.thrift.Activators;
-import org.apache.sentry.service.thrift.ServiceConstants;
 import org.apache.sentry.service.thrift.ServiceConstants.ServerConfig;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -72,7 +67,6 @@ public class TestSentryStore extends org.junit.Assert {
   final long NUM_PRIVS = 60;  // > SentryStore.PrivCleaner.NOTIFY_THRESHOLD
   private static Configuration conf = null;
   private static char[] passwd = new char[] { '1', '2', '3'};
-  private static Activator act;
 
   @BeforeClass
   public static void setup() throws Exception {
@@ -95,9 +89,6 @@ public class TestSentryStore extends org.junit.Assert {
     policyFilePath = new File(dataDir, "local_policy_file.ini");
     conf.set(ServerConfig.SENTRY_STORE_GROUP_MAPPING_RESOURCE,
         policyFilePath.getPath());
-    act = Activators.INSTANCE.create(conf);
-    conf.set(ServiceConstants.CURRENT_INCARNATION_ID_KEY,
-             act.getIncarnationId());
     sentryStore = new SentryStore(conf);
   }
 
@@ -110,23 +101,18 @@ public class TestSentryStore extends org.junit.Assert {
   }
 
   @After
-  public void after() throws SentryStandbyException {
+  public void after() {
     sentryStore.clearAllTables();
   }
 
   @AfterClass
   public static void teardown() {
-    IOUtils.cleanup(null, act);
     if (sentryStore != null) {
       sentryStore.stop();
     }
     if (dataDir != null) {
       FileUtils.deleteQuietly(dataDir);
     }
-    if (act != null) {
-      Activators.INSTANCE.remove(act);
-      act = null;
-    }
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/sentry/blob/f048132a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryStoreImportExport.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryStoreImportExport.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryStoreImportExport.java
index 98f1831..1c3a4f2 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryStoreImportExport.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryStoreImportExport.java
@@ -28,8 +28,6 @@ import java.util.Set;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.sentry.core.common.exception.SentryStandbyException;
 import org.apache.sentry.core.model.db.AccessConstants;
 import org.apache.sentry.provider.db.service.model.MSentryGroup;
 import org.apache.sentry.provider.db.service.model.MSentryPrivilege;
@@ -39,9 +37,6 @@ import org.apache.sentry.provider.db.service.thrift.TSentryGrantOption;
 import org.apache.sentry.provider.db.service.thrift.TSentryMappingData;
 import org.apache.sentry.provider.db.service.thrift.TSentryPrivilege;
 import org.apache.sentry.provider.file.PolicyFile;
-import org.apache.sentry.service.thrift.Activator;
-import org.apache.sentry.service.thrift.Activators;
-import org.apache.sentry.service.thrift.ServiceConstants;
 import org.apache.sentry.service.thrift.ServiceConstants.PrivilegeScope;
 import org.apache.sentry.service.thrift.ServiceConstants.ServerConfig;
 import org.junit.After;
@@ -60,7 +55,6 @@ public class TestSentryStoreImportExport {
   private static String[] adminGroups = { "adminGroup1" };
   private static PolicyFile policyFile;
   private static File policyFilePath;
-  private static Activator act;
   private static SentryStore sentryStore;
   private TSentryPrivilege tSentryPrivilege1;
   private TSentryPrivilege tSentryPrivilege2;
@@ -85,9 +79,6 @@ public class TestSentryStoreImportExport {
     policyFilePath = new File(dataDir, "local_policy_file.ini");
     conf.set(ServerConfig.SENTRY_STORE_GROUP_MAPPING_RESOURCE, policyFilePath.getPath());
     policyFile = new PolicyFile();
-    act = Activators.INSTANCE.create(conf);
-    conf.set(ServiceConstants.CURRENT_INCARNATION_ID_KEY,
-             act.getIncarnationId());
     sentryStore = new SentryStore(conf);
 
     String adminUser = "g1";
@@ -101,7 +92,7 @@ public class TestSentryStoreImportExport {
   }
 
   @After
-  public void clearStore() throws SentryStandbyException {
+  public void clearStore() {
     sentryStore.clearAllTables();
   }
 
@@ -138,17 +129,12 @@ public class TestSentryStoreImportExport {
 
   @AfterClass
   public static void teardown() {
-    IOUtils.cleanup(null, act);
     if (sentryStore != null) {
       sentryStore.stop();
     }
     if (dataDir != null) {
       FileUtils.deleteQuietly(dataDir);
     }
-    if (act != null) {
-      Activators.INSTANCE.remove(act);
-      act = null;
-    }
   }
 
   protected static void addGroupsToUser(String user, String... groupNames) {

http://git-wip-us.apache.org/repos/asf/sentry/blob/f048132a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryVersion.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryVersion.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryVersion.java
index e401859..a8e8a03 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryVersion.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryVersion.java
@@ -24,11 +24,7 @@ import java.io.File;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.sentry.core.common.exception.SentryNoSuchObjectException;
-import org.apache.sentry.service.thrift.Activator;
-import org.apache.sentry.service.thrift.Activators;
-import org.apache.sentry.service.thrift.ServiceConstants;
 import org.apache.sentry.service.thrift.ServiceConstants.ServerConfig;
-import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -38,7 +34,6 @@ public class TestSentryVersion {
 
   private File dataDir;
   private Configuration conf;
-  private Activator act;
 
   @Before
   public void setup() throws Exception {
@@ -47,18 +42,6 @@ public class TestSentryVersion {
     conf.set(ServerConfig.SENTRY_STORE_JDBC_URL, "jdbc:derby:;databaseName="
         + dataDir.getPath() + ";create=true");
     conf.set(ServerConfig.SENTRY_STORE_JDBC_PASS, "dummy");
-    act = Activators.INSTANCE.create(conf);
-    conf.set(ServiceConstants.CURRENT_INCARNATION_ID_KEY,
-             act.getIncarnationId());
-  }
-
-  @After
-  public void shutdown() throws Exception {
-    if (act != null) {
-      act.close();
-      Activators.INSTANCE.remove(act);
-      act = null;
-    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/sentry/blob/f048132a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/ha/TestFailover.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/ha/TestFailover.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/ha/TestFailover.java
index 57d579c..1c7cd8b 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/ha/TestFailover.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/ha/TestFailover.java
@@ -18,7 +18,6 @@
 package org.apache.sentry.tests.e2e.ha;
 
 import org.apache.sentry.tests.e2e.hive.AbstractTestWithStaticConfiguration;
-import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -40,42 +39,11 @@ public class TestFailover extends AbstractTestWithStaticConfiguration {
   /**
    * Test service failover. Run Sentry operations with shutting down one or more
    * of the services.
+   * TODO: We need rewrite the failover test for Active/Active.
    * @throws Exception
    */
   @Test
   public void testFailover() throws Exception {
-    int active = 0;
-    int passive = 1;
-    int maxTries = 10;
-    //get the active service
-    if (getSentrySrv().get(1).getActivator().isActive()) {
-      active = 1;
-      passive = 0;
-    }
-
-    Assert.assertEquals(Boolean.TRUE,getSentrySrv().get(active).getActivator().isActive());
-    Assert.assertEquals(Boolean.FALSE,getSentrySrv().get(passive).getActivator().isActive());
-
-    //stop the active service
-    getSentrySrv().get(active).becomeStandby();
-
-    for(int i = 0; i < maxTries ; i++) {
-      if(!getSentrySrv().get(active).getActivator().isActive()) {
-        break;
-      }
-      Thread.sleep(100);
-    }
-    //assert the active is now standby
-    Assert.assertEquals(Boolean.FALSE,getSentrySrv().get(active).getActivator().isActive());
-    //wait for the passive to become active
-    for(int i = 0; i < maxTries ; i++) {
-      if(getSentrySrv().get(passive).getActivator().isActive()) {
-        break;
-      }
-      Thread.sleep(100);
-    }
-    //assert that the previously passive service is now  active
-    Assert.assertEquals(Boolean.TRUE,getSentrySrv().get(passive).getActivator().isActive());
   }
 
 }