You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by sr...@apache.org on 2016/07/19 20:56:27 UTC

sentry git commit: SENTRY-1399: Integrate Fencer with SentryStore (Colin P. McCabe , Reviewed by: Hao Hao and Sravya Tirukkovalur)

Repository: sentry
Updated Branches:
  refs/heads/sentry-ha-redesign 17ed7cb7f -> 5cdd7cdfc


SENTRY-1399: Integrate Fencer with SentryStore (Colin P. McCabe , Reviewed by: Hao Hao and Sravya Tirukkovalur)

Change-Id: If73be3cb424cf16b150e67d5f8536499d86448e7


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

Branch: refs/heads/sentry-ha-redesign
Commit: 5cdd7cdfc5d363964ee6fd5336006e95ee7d8628
Parents: 17ed7cb
Author: Sravya Tirukkovalur <sr...@apache.org>
Authored: Tue Jul 19 13:55:16 2016 -0700
Committer: Sravya Tirukkovalur <sr...@apache.org>
Committed: Tue Jul 19 13:56:05 2016 -0700

----------------------------------------------------------------------
 .../exception/SentryStandbyException.java       |   4 +
 .../service/persistent/DelegateSentryStore.java |  32 ++--
 .../service/persistent/SentryStoreLayer.java    |   9 +-
 .../provider/db/service/persistent/Fencer.java  |  36 ++++-
 .../db/service/persistent/SentryStore.java      |  77 +++++++---
 .../apache/sentry/service/thrift/Activator.java |  34 ++++-
 .../sentry/service/thrift/Activators.java       |   8 +-
 .../sentry/service/thrift/SentryService.java    |   3 +-
 .../persistent/SentryStoreIntegrationBase.java  |   3 +-
 .../TestPrivilegeOperatePersistence.java        |   9 +-
 .../db/service/persistent/TestFencer.java       | 151 +++++++++++++++++++
 .../db/service/persistent/TestSentryStore.java  |   6 +-
 .../persistent/TestSentryStoreImportExport.java |   6 +-
 .../service/persistent/TestSentryVersion.java   |   3 +-
 14 files changed, 313 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sentry/blob/5cdd7cdf/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/exception/SentryStandbyException.java
----------------------------------------------------------------------
diff --git a/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/exception/SentryStandbyException.java b/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/exception/SentryStandbyException.java
index 73c7e4e..b2df699 100644
--- a/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/exception/SentryStandbyException.java
+++ b/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/exception/SentryStandbyException.java
@@ -23,6 +23,10 @@ package org.apache.sentry.core.common.exception;
 public class SentryStandbyException extends SentryUserException {
   private static final long serialVersionUID = 2162010615815L;
 
+  public SentryStandbyException(String msg, Exception e) {
+    super(msg, e);
+  }
+
   public SentryStandbyException(String msg) {
     super(msg);
   }

http://git-wip-us.apache.org/repos/asf/sentry/blob/5cdd7cdf/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 e960dcd..6f8239d 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,6 +28,7 @@ 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;
@@ -44,6 +45,8 @@ 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;
@@ -61,13 +64,16 @@ import com.google.common.collect.Sets;
  * resourceName2=cl1,resourceType2=COLUMN ) of generic privilege table
  */
 public class DelegateSentryStore implements SentryStoreLayer {
-  private SentryStore delegate;
-  private Configuration conf;
-  private Set<String> adminGroups;
-  private PrivilegeOperatePersistence privilegeOperator;
+  private final SentryStore delegate;
+  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 {
+      SentryAccessDeniedException, SentrySiteConfigurationException, IOException,
+          SentryStandbyException {
+    this.act = Activators.INSTANCE.get(conf);
     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");
@@ -100,7 +106,8 @@ public class DelegateSentryStore implements SentryStoreLayer {
 
   @Override
   public CommitContext createRole(String component, String role,
-      String requestor) throws SentryAlreadyExistsException {
+      String requestor) throws SentryAlreadyExistsException,
+          SentryStandbyException {
     return delegate.createSentryRole(role);
   }
 
@@ -110,12 +117,13 @@ public class DelegateSentryStore implements SentryStoreLayer {
    */
   @Override
   public CommitContext dropRole(String component, String role, String requestor)
-      throws SentryNoSuchObjectException {
+      throws SentryNoSuchObjectException, SentryStandbyException {
     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");
@@ -146,13 +154,15 @@ public class DelegateSentryStore implements SentryStoreLayer {
 
   @Override
   public CommitContext alterRoleAddGroups(String component, String role,
-      Set<String> groups, String requestor) throws SentryNoSuchObjectException {
+      Set<String> groups, String requestor) throws SentryNoSuchObjectException,
+          SentryStandbyException {
     return delegate.alterSentryRoleAddGroups(requestor, role, toTSentryGroups(groups));
   }
 
   @Override
   public CommitContext alterRoleDeleteGroups(String component, String role,
-      Set<String> groups, String requestor) throws SentryNoSuchObjectException {
+      Set<String> groups, String requestor) throws SentryNoSuchObjectException,
+          SentryStandbyException {
   //called to old sentryStore
     return delegate.alterSentryRoleDeleteGroups(role, toTSentryGroups(groups));
   }
@@ -166,6 +176,7 @@ 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");
@@ -197,6 +208,7 @@ 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");
@@ -240,6 +252,7 @@ public class DelegateSentryStore implements SentryStoreLayer {
     boolean rollbackTransaction = true;
     try {
       pm = openTransaction();
+      act.checkSqlFencing(pm);
 
       privilegeOperator.renamePrivilege(toTrimmedLower(component), toTrimmedLower(service),
           oldAuthorizables, newAuthorizables, requestor, pm);
@@ -263,6 +276,7 @@ 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/5cdd7cdf/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 c003965..39e55c3 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,6 +20,7 @@ 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;
@@ -40,7 +41,7 @@ public interface SentryStoreLayer {
    * @throws SentryAlreadyExistsException
    */
   CommitContext createRole(String component, String role,
-      String requestor) throws SentryAlreadyExistsException;
+      String requestor) throws SentryAlreadyExistsException, SentryStandbyException;
 
   /**
    * Drop a role
@@ -51,7 +52,7 @@ public interface SentryStoreLayer {
    * @throws SentryNoSuchObjectException
    */
   CommitContext dropRole(String component, String role,
-      String requestor) throws SentryNoSuchObjectException;
+      String requestor) throws SentryNoSuchObjectException, SentryStandbyException;
 
   /**
    * Add a role to groups.
@@ -63,7 +64,7 @@ public interface SentryStoreLayer {
    * @throws SentryNoSuchObjectException
    */
   CommitContext alterRoleAddGroups(String component, String role,
-      Set<String> groups, String requestor) throws SentryNoSuchObjectException;
+      Set<String> groups, String requestor) throws SentryNoSuchObjectException, SentryStandbyException;
 
   /**
    * Delete a role from groups.
@@ -75,7 +76,7 @@ public interface SentryStoreLayer {
    * @throws SentryNoSuchObjectException
    */
   CommitContext alterRoleDeleteGroups(String component, String role,
-      Set<String> groups, String requestor) throws SentryNoSuchObjectException;
+      Set<String> groups, String requestor) throws SentryNoSuchObjectException, SentryStandbyException;
 
   /**
    * Grant a privilege to role.

http://git-wip-us.apache.org/repos/asf/sentry/blob/5cdd7cdf/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/Fencer.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/Fencer.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/Fencer.java
index 14cdde3..77aaac5 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/Fencer.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/Fencer.java
@@ -20,6 +20,7 @@ package org.apache.sentry.provider.db.service.persistent;
 
 import java.util.List;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
 
 import javax.jdo.JDOException;
@@ -29,6 +30,7 @@ import javax.jdo.PersistenceManagerFactory;
 import javax.jdo.Query;
 import javax.jdo.Transaction;
 
+import org.apache.sentry.core.common.exception.SentryStandbyException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -76,7 +78,7 @@ public class Fencer {
   private final static String SENTRY_FENCE_TABLE_BASE = "SENTRY_FENCE";
 
   /**
-   * The update log table name, including the incarnation ID.
+   * The fencer table name, including the incarnation ID.
    */
   private final String tableIncarnationName;
 
@@ -86,7 +88,7 @@ public class Fencer {
   private final SqlAccessor sql;
 
   /**
-   * Create an accessor for the update log.
+   * Create the Fencer.
    *
    * @param incarnationId     The ID of the current sentry daemon incarnation.
    * @param pmf               The PersistenceManagerFactory to use.
@@ -95,12 +97,13 @@ public class Fencer {
     this.tableIncarnationName = String.
         format("%s_%s", SENTRY_FENCE_TABLE_BASE, incarnationId);
     this.sql = SqlAccessor.get(pmf);
+    LOGGER.info("Loaded Fencer for " + sql.getDatabaseName());
   }
 
   /**
    * Finds the name of the fencing table.<p/>
    *
-   * The name of the update log table will always begin with SENTRY_UPDATE_LOG,
+   * The name of the fencer table will always begin with SENTRY_FENCE,
    * but it may have the ID of a previous sentry incarnation tacked on to it.
    *
    * @return the current name of the update log table, or null if there is none.
@@ -228,15 +231,32 @@ public class Fencer {
   }
 
   /**
-   * Attempt to append an UpdateLogEntry to the update log.
+   * Verify that the fencing table still exists by running a query on it.
    */
-  void verify(PersistenceManager pm) {
-    Query query = pm.newQuery(SqlAccessor.JDO_SQL_ESCAPE,
-        sql.getFetchAllRowsSql(tableIncarnationName));
-    query.execute();
+  public void checkSqlFencing(PersistenceManager pm)
+      throws SentryStandbyException {
+    try {
+      Query query = pm.newQuery(SqlAccessor.JDO_SQL_ESCAPE,
+          sql.getFetchAllRowsSql(tableIncarnationName));
+      query.execute();
+    } catch (JDOException e) {
+      throw new SentryStandbyException("Failed to verify that " +
+          "the daemon was still active", e);
+    }
   }
 
   String getTableIncarnationName() {
     return tableIncarnationName;
   }
+
+  /**
+   * Rename the update log table so that fencing is no longer active.
+   * This is only used in unit tests currently.
+   */
+  @VisibleForTesting
+  public void unfence(PersistenceManagerFactory pmf) {
+    renameTable(pmf, tableIncarnationName, SENTRY_FENCE_TABLE_BASE);
+    LOGGER.info("Renamed " + tableIncarnationName + " to "  +
+        SENTRY_FENCE_TABLE_BASE);
+  }
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/5cdd7cdf/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 6e367e5..e2b92f9 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,6 +46,7 @@ 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;
@@ -99,7 +100,7 @@ import com.google.common.collect.Sets;
 public class SentryStore {
   private static final UUID SERVER_UUID = UUID.randomUUID();
   private static final Logger LOGGER = LoggerFactory
-          .getLogger(SentryStore.class);
+      .getLogger(SentryStore.class);
 
   public static final String NULL_COL = "__NULL__";
   public static int INDEX_GROUP_ROLES_MAP = 0;
@@ -148,7 +149,7 @@ public class SentryStore {
     // it falls back to reading directly from sentry-site.xml
     char[] passTmp = conf.getPassword(ServerConfig.SENTRY_STORE_JDBC_PASS);
     String pass = null;
-    if(passTmp != null) {
+    if (passTmp != null) {
       pass = new String(passTmp);
     } else {
       throw new SentrySiteConfigurationException("Error reading " + ServerConfig.SENTRY_STORE_JDBC_PASS);
@@ -176,7 +177,8 @@ public class SentryStore {
 
   public SentryStore(Configuration conf)
       throws SentryNoSuchObjectException, SentryAccessDeniedException,
-          SentrySiteConfigurationException, IOException {
+          SentrySiteConfigurationException, IOException,
+              SentryStandbyException {
     this.act = Activators.INSTANCE.get(conf);
     commitSequenceId = 0;
     this.conf = conf;
@@ -184,7 +186,7 @@ public class SentryStore {
     boolean checkSchemaVersion = conf.get(
         ServerConfig.SENTRY_VERIFY_SCHEM_VERSION,
         ServerConfig.SENTRY_VERIFY_SCHEM_VERSION_DEFAULT).equalsIgnoreCase(
-            "true");
+        "true");
     if (!checkSchemaVersion) {
       prop.setProperty("datanucleus.schema.autoCreateAll", "true");
       prop.setProperty("datanucleus.autoCreateSchema", "true");
@@ -196,8 +198,8 @@ public class SentryStore {
     // Kick off the thread that cleans orphaned privileges (unless told not to)
     privCleaner = this.new PrivCleaner();
     if (conf.get(ServerConfig.SENTRY_STORE_ORPHANED_PRIVILEGE_REMOVAL,
-            ServerConfig.SENTRY_STORE_ORPHANED_PRIVILEGE_REMOVAL_DEFAULT)
-            .equalsIgnoreCase("true")) {
+        ServerConfig.SENTRY_STORE_ORPHANED_PRIVILEGE_REMOVAL_DEFAULT)
+        .equalsIgnoreCase("true")) {
       privCleanerThread = new Thread(privCleaner);
       privCleanerThread.start();
     }
@@ -205,7 +207,8 @@ public class SentryStore {
 
   // ensure that the backend DB schema is set
   public void verifySentryStoreSchema(boolean checkVersion)
-          throws SentryNoSuchObjectException, SentryAccessDeniedException {
+      throws SentryNoSuchObjectException, SentryAccessDeniedException,
+          SentryStandbyException {
     if (!checkVersion) {
       setSentryVersion(SentryStoreSchemaInfo.getSentryVersion(),
           "Schema version set implicitly");
@@ -214,8 +217,8 @@ public class SentryStore {
       if (!SentryStoreSchemaInfo.getSentryVersion().equals(currentVersion)) {
         throw new SentryAccessDeniedException(
             "The Sentry store schema version " + currentVersion
-            + " is different from distribution version "
-            + SentryStoreSchemaInfo.getSentryVersion());
+                + " is different from distribution version "
+                + SentryStoreSchemaInfo.getSentryVersion());
       }
     }
   }
@@ -241,10 +244,10 @@ public class SentryStore {
    * instance when we create a new transaction. We create a new transaction
    * for every store API since we want that unit of work to behave as a
    * transaction.
-   *
+   * <p/>
    * Note that there's only one instance of PersistenceManagerFactory object
    * for the service.
-   *
+   * <p/>
    * Synchronized because we obtain persistence manager
    */
   public synchronized PersistenceManager openTransaction() {
@@ -320,11 +323,12 @@ public class SentryStore {
    * @throws SentryAlreadyExistsException
    */
   public CommitContext createSentryRole(String roleName)
-      throws SentryAlreadyExistsException {
+      throws SentryAlreadyExistsException, SentryStandbyException {
     boolean rollbackTransaction = true;
     PersistenceManager pm = null;
     try {
       pm = openTransaction();
+      act.checkSqlFencing(pm);
       createSentryRoleCore(pm, roleName);
       CommitContext commit = commitUpdateTransaction(pm);
       rollbackTransaction = false;
@@ -412,11 +416,12 @@ public class SentryStore {
   }
 
   @VisibleForTesting
-  void clearAllTables() {
+  void clearAllTables() throws SentryStandbyException {
     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();
@@ -445,6 +450,7 @@ 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);
@@ -536,6 +542,7 @@ 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);
@@ -800,11 +807,12 @@ public class SentryStore {
   }
 
   public CommitContext dropSentryRole(String roleName)
-      throws SentryNoSuchObjectException {
+      throws SentryNoSuchObjectException, SentryStandbyException {
     boolean rollbackTransaction = true;
     PersistenceManager pm = null;
     try {
       pm = openTransaction();
+      act.checkSqlFencing(pm);
       dropSentryRoleCore(pm, roleName);
       CommitContext commit = commitUpdateTransaction(pm);
       rollbackTransaction = false;
@@ -839,11 +847,12 @@ public class SentryStore {
 
   public CommitContext alterSentryRoleAddGroups(String grantorPrincipal, String roleName,
       Set<TSentryGroup> groupNames)
-          throws SentryNoSuchObjectException {
+      throws SentryNoSuchObjectException, SentryStandbyException {
     boolean rollbackTransaction = true;
     PersistenceManager pm = null;
     try {
       pm = openTransaction();
+      act.checkSqlFencing(pm);
       alterSentryRoleAddGroupsCore(pm, roleName, groupNames);
       CommitContext commit = commitUpdateTransaction(pm);
       rollbackTransaction = false;
@@ -885,11 +894,13 @@ public class SentryStore {
   }
 
   public CommitContext alterSentryRoleAddUsers(String roleName,
-      Set<String> userNames) throws SentryNoSuchObjectException {
+      Set<String> userNames) throws SentryNoSuchObjectException,
+          SentryStandbyException {
     boolean rollbackTransaction = true;
     PersistenceManager pm = null;
     try {
       pm = openTransaction();
+      act.checkSqlFencing(pm);
       alterSentryRoleAddUsersCore(pm, roleName, userNames);
       CommitContext commit = commitUpdateTransaction(pm);
       rollbackTransaction = false;
@@ -927,12 +938,13 @@ public class SentryStore {
   }
 
   public CommitContext alterSentryRoleDeleteUsers(String roleName, Set<String> userNames)
-      throws SentryNoSuchObjectException {
+      throws SentryNoSuchObjectException, SentryStandbyException {
     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);
@@ -964,12 +976,13 @@ public class SentryStore {
 
   public CommitContext alterSentryRoleDeleteGroups(String roleName,
       Set<TSentryGroup> groupNames)
-          throws SentryNoSuchObjectException {
+      throws SentryNoSuchObjectException, SentryStandbyException {
     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");
@@ -1612,7 +1625,8 @@ public class SentryStore {
   }
 
   public void setSentryVersion(String newVersion, String verComment)
-      throws SentryNoSuchObjectException, SentryAccessDeniedException {
+      throws SentryNoSuchObjectException, SentryAccessDeniedException,
+          SentryStandbyException {
     MSentryVersion mVersion;
     boolean rollbackTransaction = true;
     PersistenceManager pm = null;
@@ -1631,6 +1645,7 @@ public class SentryStore {
     mVersion.setVersionComment(verComment);
     try {
       pm = openTransaction();
+      act.checkSqlFencing(pm);
       pm.makePersistent(mVersion);
       rollbackTransaction = false;
       commitTransaction(pm);
@@ -1680,13 +1695,15 @@ public class SentryStore {
    * Drop given privilege from all roles
    */
   public void dropPrivilege(TSentryAuthorizable tAuthorizable)
-      throws SentryNoSuchObjectException, SentryInvalidInputException {
+      throws SentryNoSuchObjectException, SentryInvalidInputException,
+          SentryStandbyException {
     PersistenceManager pm = null;
     boolean rollbackTransaction = true;
 
     TSentryPrivilege tPrivilege = toSentryPrivilege(tAuthorizable);
     try {
       pm = openTransaction();
+      act.checkSqlFencing(pm);
 
       if (isMultiActionsSupported(tPrivilege)) {
         for (String privilegeAction : ALL_ACTIONS) {
@@ -1717,7 +1734,8 @@ public class SentryStore {
    */
   public void renamePrivilege(TSentryAuthorizable tAuthorizable,
       TSentryAuthorizable newTAuthorizable)
-      throws SentryNoSuchObjectException, SentryInvalidInputException {
+          throws SentryNoSuchObjectException, SentryInvalidInputException,
+              SentryStandbyException {
     PersistenceManager pm = null;
     boolean rollbackTransaction = true;
 
@@ -1726,6 +1744,7 @@ 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) {
@@ -2023,13 +2042,15 @@ public class SentryStore {
   }
 
   public CommitContext createAuthzPathsMapping(String hiveObj,
-      Set<String> paths) throws SentryNoSuchObjectException, SentryAlreadyExistsException {
+      Set<String> paths) throws SentryNoSuchObjectException,
+          SentryAlreadyExistsException, SentryStandbyException {
 
     boolean rollbackTransaction = true;
     PersistenceManager pm = null;
 
     try {
       pm = openTransaction();
+      act.checkSqlFencing(pm);
       createAuthzPathsMappingCore(pm, hiveObj, paths);
       CommitContext commit = commitUpdateTransaction(pm);
       rollbackTransaction = false;
@@ -2079,7 +2100,9 @@ public class SentryStore {
     private static final int NOTIFY_THRESHOLD = 50;
 
     // How many times we've been notified; reset to zero after orphan removal
-    private int currentNotifies = 0;
+    // This begins at the NOTIFY_THRESHOLD, so that we clear any potential
+    // orphans on startup.
+    private int currentNotifies = NOTIFY_THRESHOLD;
 
     // Internal state for threads
     private boolean exitRequired = false;
@@ -2123,7 +2146,9 @@ public class SentryStore {
           lock.unlock();
         }
         try {
-          removeOrphanedPrivileges();
+          if (act.isActive()) {
+            removeOrphanedPrivileges();
+          }
         } catch (Exception e) {
           LOGGER.warn("Privilege cleaning thread encountered an error: " +
                   e.getMessage());
@@ -2186,7 +2211,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() {
+    private void removeOrphanedPrivileges() throws SentryStandbyException {
       final String privDB = "SENTRY_DB_PRIVILEGE";
       final String privId = "DB_PRIVILEGE_ID";
       final String mapDB = "SENTRY_ROLE_DB_PRIVILEGE_MAP";
@@ -2207,6 +2232,7 @@ 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();
@@ -2589,6 +2615,7 @@ 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/5cdd7cdf/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 0b7ddf5..730dbb1 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
@@ -23,12 +23,14 @@ import java.io.IOException;
 import java.util.Properties;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.sentry.core.common.exception.SentryStandbyException;
 import org.apache.sentry.provider.db.service.persistent.Fencer;
 import org.apache.sentry.provider.db.service.persistent.SentryStore;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import javax.jdo.JDOHelper;
+import javax.jdo.PersistenceManager;
 import javax.jdo.PersistenceManagerFactory;
 
 /**
@@ -60,8 +62,11 @@ public class Activator implements Closeable {
 
   /**
    * True if the Activator is active.
+   *
+   * This variable can be read without taking the lock, but must not be
+   * written unless we hold the Activator lock.
    */
-  private boolean active;
+  private volatile boolean active = false;
 
   public Activator(Configuration conf) throws Exception {
     Properties props = SentryStore.getDataNucleusProperties(conf);
@@ -69,7 +74,6 @@ public class Activator implements Closeable {
     this.handler = new TransitionHandler();
     this.leaderStatus = new LeaderStatus(handler, conf);
     this.fencer = new Fencer(this.leaderStatus.getIncarnationId(), pmf);
-    this.active = false;
     this.leaderStatus.start();
   }
 
@@ -102,11 +106,35 @@ public class Activator implements Closeable {
     }
   }
 
-  synchronized boolean isActive() {
+  /**
+   * Returns true if this Activator considers itself active.
+   * Note that you must still use checkSqlFencing or another
+   * means of fencing when performing modification operations.
+   */
+  public boolean isActive() {
     return active;
   }
 
   public synchronized String getIncarnationId() {
     return leaderStatus.getIncarnationId();
   }
+
+  public Fencer getFencer() {
+    return fencer;
+  }
+
+  /**
+   * Verify that the current SQL transaction is safe. 
+   */
+  public void checkSqlFencing(PersistenceManager pm)
+        throws SentryStandbyException {
+    // Before invoking the fencer, first check if we believe that we are
+    // active.  This avoids wasted effort.
+    if (!active) {
+      throw new SentryStandbyException("The daemon is not active");
+    }
+    // If we believe that we are active, add a query to the current transaction
+    // which will confirm that fact.
+    fencer.checkSqlFencing(pm);
+  }
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/5cdd7cdf/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/Activators.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/Activators.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/Activators.java
index 37b0219..2926eeb 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/Activators.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/Activators.java
@@ -41,6 +41,12 @@ public enum Activators {
     acts.put(act.getIncarnationId(), act);
   }
 
+  public Activator create(Configuration conf) throws Exception {
+    Activator act = new Activator(conf);
+    put(act);
+    return act;
+  }
+
   public Activator get(Configuration conf) {
     String key = conf.get(SentryConstants.CURRENT_INCARNATION_ID_KEY);
     if (key == null) {
@@ -50,7 +56,7 @@ public enum Activators {
     return get(key);
   }
 
-  public synchronized Activator get(String incarnationId) {
+  private synchronized Activator get(String incarnationId) {
     Activator act = acts.get(incarnationId);
     if (act == null) {
       throw new RuntimeException("No activator found with " +

http://git-wip-us.apache.org/repos/asf/sentry/blob/5cdd7cdf/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 531ab35..51dde0e 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
@@ -153,10 +153,9 @@ public class SentryService implements Callable {
             + (count++));
       }
     });
-    this.act = new Activator(conf);
+    this.act = Activators.INSTANCE.create(conf);
     conf.set(SentryConstants.CURRENT_INCARNATION_ID_KEY,
         this.act.getIncarnationId());
-    Activators.INSTANCE.put(act);
     webServerPort = conf.getInt(ServerConfig.SENTRY_WEB_PORT, ServerConfig.SENTRY_WEB_PORT_DEFAULT);
     status = Status.NOT_STARTED;
   }

http://git-wip-us.apache.org/repos/asf/sentry/blob/5cdd7cdf/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 5999580..c52197f 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
@@ -62,9 +62,8 @@ public abstract class SentryStoreIntegrationBase {
     policyFilePath = new File(Files.createTempDir(), "local_policy_file.ini");
     conf.set(ServerConfig.SENTRY_STORE_GROUP_MAPPING_RESOURCE,
         policyFilePath.getPath());
-    act = new Activator(conf);
+    act = Activators.INSTANCE.create(conf);
 		conf.set(ServiceConstants.CURRENT_INCARNATION_ID_KEY, act.getIncarnationId());
-    Activators.INSTANCE.put(act);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/sentry/blob/5cdd7cdf/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 7c66db4..fd10a7a 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
@@ -989,10 +989,9 @@ 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 = new Activator(confCopy);
+    Activator act = Activators.INSTANCE.create(confCopy);
     confCopy.set(ServiceConstants.CURRENT_INCARNATION_ID_KEY,
                  act.getIncarnationId());
-    Activators.INSTANCE.put(act);
    SentryStoreLayer store = new DelegateSentryStore(confCopy);
     testGrantPrivilege(store, externalComponent);
     act.close();
@@ -1005,10 +1004,9 @@ 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 = new Activator(confCopy);
+    Activator act = Activators.INSTANCE.create(confCopy);
     confCopy.set(ServiceConstants.CURRENT_INCARNATION_ID_KEY,
                  act.getIncarnationId());
-    Activators.INSTANCE.put(act);
     SentryStoreLayer store = new DelegateSentryStore(confCopy);
     testGrantPrivilege(store, externalComponent);
     act.close();
@@ -1021,10 +1019,9 @@ 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 = new Activator(confCopy);
+    Activator act = Activators.INSTANCE.create(confCopy);
     confCopy.set(ServiceConstants.CURRENT_INCARNATION_ID_KEY,
                  act.getIncarnationId());
-    Activators.INSTANCE.put(act);
     SentryStoreLayer store = new DelegateSentryStore(confCopy);
     testGrantPrivilege(store, externalComponent);
     act.close();

http://git-wip-us.apache.org/repos/asf/sentry/blob/5cdd7cdf/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
new file mode 100644
index 0000000..42b67ba
--- /dev/null
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestFencer.java
@@ -0,0 +1,151 @@
+/**
+ * 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.service.persistent;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+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;
+
+import com.google.common.io.Files;
+
+import javax.jdo.JDOHelper;
+import javax.jdo.PersistenceManagerFactory;
+
+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;
+
+    DatabaseContext() {
+      this.conf = new Configuration();
+      this.dataDir = new File(Files.createTempDir(), "sentry_policy_db");
+      this.conf.set(ServerConfig.SENTRY_STORE_JDBC_URL,
+          "jdbc:derby:;databaseName=" + dataDir.getPath() + ";create=true");
+      this.conf.set(ServerConfig.SENTRY_STORE_JDBC_PASS, "dummy");
+    }
+
+    @Override
+    public void close() throws IOException {
+      FileUtils.deleteQuietly(dataDir);
+    }
+
+    public Configuration getConf() {
+      return conf;
+    }
+  }
+
+  @Test(timeout = 60000)
+  public void testInvokingFencer() throws Exception {
+    DatabaseContext dbCtx = null;
+    PersistenceManagerFactory pmf = null;
+    try {
+      dbCtx = new DatabaseContext();
+      Properties prop = SentryStore.getDataNucleusProperties(dbCtx.getConf());
+      pmf = JDOHelper.getPersistenceManagerFactory(prop);
+      Fencer fencer = new Fencer("abc", pmf);
+      fencer.fence(pmf);
+      fencer.unfence(pmf);
+    } finally {
+      IOUtils.cleanup(null, dbCtx);
+      if (pmf != null) {
+        try {
+          pmf.close();
+        } catch (Exception e) {
+          LOGGER.error("error closing pmf" , e);
+        }
+      }
+    }
+  }
+
+  @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/5cdd7cdf/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 6e00505..0634206 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
@@ -32,6 +32,7 @@ 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;
@@ -94,10 +95,9 @@ 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 = new Activator(conf);
+    act = Activators.INSTANCE.create(conf);
     conf.set(ServiceConstants.CURRENT_INCARNATION_ID_KEY,
              act.getIncarnationId());
-    Activators.INSTANCE.put(act);
     sentryStore = new SentryStore(conf);
   }
 
@@ -110,7 +110,7 @@ public class TestSentryStore extends org.junit.Assert {
   }
 
   @After
-  public void after() {
+  public void after() throws SentryStandbyException {
     sentryStore.clearAllTables();
   }
 

http://git-wip-us.apache.org/repos/asf/sentry/blob/5cdd7cdf/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 fc39658..98f1831 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
@@ -29,6 +29,7 @@ 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;
@@ -84,10 +85,9 @@ 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 = new Activator(conf);
+    act = Activators.INSTANCE.create(conf);
     conf.set(ServiceConstants.CURRENT_INCARNATION_ID_KEY,
              act.getIncarnationId());
-    Activators.INSTANCE.put(act);
     sentryStore = new SentryStore(conf);
 
     String adminUser = "g1";
@@ -101,7 +101,7 @@ public class TestSentryStoreImportExport {
   }
 
   @After
-  public void clearStore() {
+  public void clearStore() throws SentryStandbyException {
     sentryStore.clearAllTables();
   }
 

http://git-wip-us.apache.org/repos/asf/sentry/blob/5cdd7cdf/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 cf7ca8e..e401859 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
@@ -47,10 +47,9 @@ 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 = new Activator(conf);
+    act = Activators.INSTANCE.create(conf);
     conf.set(ServiceConstants.CURRENT_INCARNATION_ID_KEY,
              act.getIncarnationId());
-    Activators.INSTANCE.put(act);
   }
 
   @After