You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by ls...@apache.org on 2014/11/25 08:54:18 UTC

incubator-sentry git commit: SENTRY-140: Orphaned privileges should be garbage collected (Mike Yoder via Lenni Kuff)

Repository: incubator-sentry
Updated Branches:
  refs/heads/master 7701fd8e3 -> ac919b28d


SENTRY-140: Orphaned privileges should be garbage collected (Mike Yoder via Lenni Kuff)


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

Branch: refs/heads/master
Commit: ac919b28dfb12730b22c27de0209d3da7897fccc
Parents: 7701fd8
Author: Lenni Kuff <ls...@cloudera.com>
Authored: Mon Nov 24 23:53:39 2014 -0800
Committer: Lenni Kuff <ls...@cloudera.com>
Committed: Mon Nov 24 23:53:39 2014 -0800

----------------------------------------------------------------------
 .../db/service/persistent/SentryStore.java      | 236 ++++++++++++++++++-
 .../sentry/service/thrift/ServiceConstants.java |   5 +-
 .../db/service/persistent/TestSentryStore.java  | 102 ++++++++
 3 files changed, 341 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/ac919b28/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 3615661..073bb33 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
@@ -31,6 +31,9 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
 import java.util.UUID;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
 
 import javax.jdo.FetchGroup;
 import javax.jdo.JDODataStoreException;
@@ -76,6 +79,8 @@ import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * SentryStore is the data access object for Sentry data. Strings
@@ -84,6 +89,8 @@ 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);
 
   public static String NULL_COL = "__NULL__";
   static final String DEFAULT_DATA_DIR = "sentry_policy_db";
@@ -98,6 +105,8 @@ public class SentryStore {
   private long commitSequenceId;
   private final PersistenceManagerFactory pmf;
   private Configuration conf;
+  private PrivCleaner privCleaner = null;
+  private Thread privCleanerThread = null;
 
   public SentryStore(Configuration conf) throws SentryNoSuchObjectException,
   SentryAccessDeniedException {
@@ -136,8 +145,22 @@ public class SentryStore {
       prop.setProperty("datanucleus.autoCreateSchema", "true");
       prop.setProperty("datanucleus.fixedDatastore", "false");
     }
+
+    // Disallow operations outside of transactions
+    prop.setProperty("datanucleus.NontransactionalRead", "false");
+    prop.setProperty("datanucleus.NontransactionalWrite", "false");
+
     pmf = JDOHelper.getPersistenceManagerFactory(prop);
     verifySentryStoreSchema(conf, checkSchemaVersion);
+
+    // 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")) {
+      privCleanerThread = new Thread(privCleaner);
+      privCleanerThread.start();
+    }
   }
 
   // ensure that the backend DB schema is set
@@ -159,6 +182,14 @@ public class SentryStore {
   }
 
   public synchronized void stop() {
+    if (privCleanerThread != null) {
+      privCleaner.exit();
+      try {
+        privCleanerThread.join();
+      } catch (InterruptedException e) {
+        // Ignore...
+      }
+    }
     if (pmf != null) {
       pmf.close();
     }
@@ -315,6 +346,16 @@ public class SentryStore {
     };
   }
 
+  /**
+   * Lets the test code know how many privs are in the db, so that we know
+   * if they are in fact being cleaned up when not being referenced any more.
+   * @return The number of rows in the db priv table.
+   */
+  @VisibleForTesting
+  long countMSentryPrivileges() {
+    return getCount(MSentryPrivilege.class);
+  }
+
   public CommitContext alterSentryRoleGrantPrivilege(String grantorPrincipal,
       String roleName, TSentryPrivilege privilege)
       throws SentryUserException {
@@ -371,10 +412,12 @@ public class SentryStore {
           MSentryPrivilege mInsert = getMSentryPrivilege(tNotAll, pm);
           if ((mSelect != null) && (mRole.getPrivileges().contains(mSelect))) {
             mSelect.removeRole(mRole);
+            privCleaner.incPrivRemoval();
             pm.makePersistent(mSelect);
           }
           if ((mInsert != null) && (mRole.getPrivileges().contains(mInsert))) {
             mInsert.removeRole(mRole);
+            privCleaner.incPrivRemoval();
             pm.makePersistent(mInsert);
           }
         } else {
@@ -484,6 +527,7 @@ public class SentryStore {
 
     if (requestedPrivToRevoke.getAction().equalsIgnoreCase("ALL") || requestedPrivToRevoke.getAction().equalsIgnoreCase("*")) {
       persistedPriv.removeRole(mRole);
+      privCleaner.incPrivRemoval();
       pm.makePersistent(persistedPriv);
     } else if (requestedPrivToRevoke.getAction().equalsIgnoreCase(AccessConstants.SELECT)
         && (!currentPrivilege.getAction().equalsIgnoreCase(AccessConstants.INSERT))) {
@@ -499,12 +543,14 @@ public class SentryStore {
       throws SentryInvalidInputException {
     // If table / URI, remove ALL
     persistedPriv.removeRole(mRole);
+    privCleaner.incPrivRemoval();
     pm.makePersistent(persistedPriv);
 
     currentPrivilege.setAction(AccessConstants.ALL);
     persistedPriv = getMSentryPrivilege(convertToTSentryPrivilege(currentPrivilege), pm);
     if ((persistedPriv != null)&&(mRole.getPrivileges().contains(persistedPriv))) {
       persistedPriv.removeRole(mRole);
+      privCleaner.incPrivRemoval();
       pm.makePersistent(persistedPriv);
 
       currentPrivilege.setAction(addAction);
@@ -660,8 +706,9 @@ public class SentryStore {
         throw new SentryNoSuchObjectException("Role " + roleName);
       } else {
         pm.retrieve(sentryRole);
+        int numPrivs = sentryRole.getPrivileges().size();
         sentryRole.removePrivileges();
-
+        privCleaner.incPrivRemoval(numPrivs);
         pm.deletePersistent(sentryRole);
       }
       CommitContext commit = commitUpdateTransaction(pm);
@@ -1676,4 +1723,191 @@ public class SentryStore {
     }
   }
 
+  /**
+   * This thread exists to clean up "orphaned" privilege rows in the database.
+   * These rows aren't removed automatically due to the fact that there is
+   * a many-to-many mapping between the roles and privileges, and the
+   * detection and removal of orphaned privileges is a wee bit involved.
+   * This thread hangs out until notified by the parent (the outer class)
+   * and then runs a custom SQL statement that detects and removes orphans.
+   */
+  private class PrivCleaner implements Runnable {
+    // Kick off priv orphan removal after this many notifies
+    private static final int NOTIFY_THRESHOLD = 50;
+
+    // How many times we've been notified; reset to zero after orphan removal
+    private int currentNotifies = 0;
+
+    // Internal state for threads
+    private boolean exitRequired = false;
+
+    // This lock and condition are needed to implement a way to drop the
+    // lock inside a while loop, and not hold the lock across the orphan
+    // removal.
+    private final Lock lock = new ReentrantLock();
+    private final Condition cond = lock.newCondition();
+
+    /**
+     * Waits in a loop, running the orphan removal function when notified.
+     * Will exit after exitRequired is set to true by exit().  We are careful
+     * to not hold our lock while removing orphans; that operation might
+     * take a long time.  There's also the matter of lock ordering.  Other
+     * threads start a transaction first, and then grab our lock; this thread
+     * grabs the lock and then starts a transaction.  Handling this correctly
+     * requires explicit locking/unlocking through the loop.
+     */
+    public void run() {
+      while (true) {
+        lock.lock();
+        try {
+          // Check here in case this was set during removeOrphanedPrivileges()
+          if (exitRequired) {
+            return;
+          }
+          while (currentNotifies <= NOTIFY_THRESHOLD) {
+            try {
+              cond.await();
+            } catch (InterruptedException e) {
+              // Interrupted
+            }
+            // Check here in case this was set while waiting
+            if (exitRequired) {
+              return;
+            }
+          }
+          currentNotifies = 0;
+        } finally {
+          lock.unlock();
+        }
+        try {
+          removeOrphanedPrivileges();
+        } catch (Exception e) {
+          LOGGER.warn("Privilege cleaning thread encountered an error: " +
+                  e.getMessage());
+        }
+      }
+    }
+
+    /**
+     * This is called when a privilege is removed from a role.  This may
+     * or may not mean that the privilege needs to be removed from the
+     * database; there may be more references to it from other roles.
+     * As a result, we'll lazily run the orphan cleaner every
+     * NOTIFY_THRESHOLD times this routine is called.
+     * @param numDeletions The number of potentially orphaned privileges
+     */
+    public void incPrivRemoval(int numDeletions) {
+      if (privCleanerThread != null) {
+        lock.lock();
+        currentNotifies += numDeletions;
+        if (currentNotifies > NOTIFY_THRESHOLD) {
+          cond.signal();
+        }
+        lock.unlock();
+      }
+    }
+
+    /**
+     * Simple form of incPrivRemoval when only one privilege is deleted.
+     */
+    public void incPrivRemoval() {
+      incPrivRemoval(1);
+    }
+
+    /**
+     * Tell this thread to exit. Safe to call multiple times, as it just
+     * notifies the run() loop to finish up.
+     */
+    public void exit() {
+      if (privCleanerThread != null) {
+        lock.lock();
+        try {
+          exitRequired = true;
+          cond.signal();
+        } finally {
+          lock.unlock();
+        }
+      }
+    }
+
+    /**
+     * Run a SQL query to detect orphaned privileges, and then delete
+     * each one.  This is complicated by the fact that datanucleus does
+     * not seem to play well with the mix between a direct SQL query
+     * and operations on the database.  The solution that seems to work
+     * is to split the operation into two transactions: the first is
+     * just a read for privileges that look like they're orphans, the
+     * 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() {
+      final String privDB = "SENTRY_DB_PRIVILEGE";
+      final String privId = "DB_PRIVILEGE_ID";
+      final String mapDB = "SENTRY_ROLE_DB_PRIVILEGE_MAP";
+      final String privFilter =
+              "select " + privId +
+              " from " + privDB + " p" +
+              " where not exists (" +
+                  " select 1 from " + mapDB + " d" +
+                  " where p." + privId + " != d." + privId +
+              " )";
+      boolean rollback = true;
+      int orphansRemoved = 0;
+      ArrayList<Object> idList = new ArrayList<Object>();
+      PersistenceManager pm = pmf.getPersistenceManager();
+
+      // Transaction 1: Perform a SQL query to get things that look like orphans
+      try {
+        Transaction transaction = pm.currentTransaction();
+        transaction.begin();
+        transaction.setRollbackOnly();  // Makes the tx read-only
+        Query query = pm.newQuery("javax.jdo.query.SQL", privFilter);
+        query.setClass(MSentryPrivilege.class);
+        List<MSentryPrivilege> results = (List<MSentryPrivilege>) query.execute();
+        for (MSentryPrivilege orphan : results) {
+          idList.add(pm.getObjectId(orphan));
+        }
+        transaction.rollback();
+        rollback = false;
+      } finally {
+        if (rollback && pm.currentTransaction().isActive()) {
+          pm.currentTransaction().rollback();
+        } else {
+          LOGGER.debug("Found {} potential orphans", idList.size());
+        }
+      }
+
+      if (idList.isEmpty()) {
+        pm.close();
+        return;
+      }
+
+      Preconditions.checkState(!rollback);
+
+      // Transaction 2: For each potential orphan, verify it's really an
+      // orphan and delete it if so
+      rollback = true;
+      try {
+        Transaction transaction = pm.currentTransaction();
+        transaction.begin();
+        pm.refreshAll();  // Try to ensure we really have correct objects
+        for (Object id : idList) {
+          MSentryPrivilege priv = (MSentryPrivilege) pm.getObjectById(id);
+          if (priv.getRoles().isEmpty()) {
+            pm.deletePersistent(priv);
+            orphansRemoved++;
+          }
+        }
+        transaction.commit();
+        pm.close();
+        rollback = false;
+      } finally {
+        if (rollback) {
+          rollbackTransaction(pm);
+        } else {
+          LOGGER.debug("Cleaned up {} orphaned privileges", orphansRemoved);
+        }
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/ac919b28/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/ServiceConstants.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/ServiceConstants.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/ServiceConstants.java
index 03ed378..eb92c1e 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/ServiceConstants.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/ServiceConstants.java
@@ -99,13 +99,16 @@ public class ServiceConstants {
 
     public static final String SENTRY_SERVICE_NAME = "sentry.service.name";
     public static final String SENTRY_SERVICE_NAME_DEFAULT = "Sentry-Service";
-    
+
     public static final String SENTRY_STORE_GROUP_MAPPING = "sentry.store.group.mapping";
     public static final String SENTRY_STORE_GROUP_MAPPING_RESOURCE = "sentry.store.group.mapping.resource";
     public static final String SENTRY_STORE_HADOOP_GROUP_MAPPING = "org.apache.sentry.provider.common.HadoopGroupMappingService";
     public static final String SENTRY_STORE_LOCAL_GROUP_MAPPING = "org.apache.sentry.provider.file.LocalGroupMappingService";
     public static final String SENTRY_STORE_GROUP_MAPPING_DEFAULT = SENTRY_STORE_HADOOP_GROUP_MAPPING;
 
+    public static final String SENTRY_STORE_ORPHANED_PRIVILEGE_REMOVAL = "sentry.store.orphaned.privilege.removal";
+    public static final String SENTRY_STORE_ORPHANED_PRIVILEGE_REMOVAL_DEFAULT = "true";
+
     public static final ImmutableMap<String, String> SENTRY_STORE_DEFAULTS =
         ImmutableMap.<String, String>builder()
     .put("datanucleus.connectionPoolingType", "BoneCP")

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/ac919b28/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 70917b7..e8be81f 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
@@ -59,6 +59,7 @@ public class TestSentryStore {
   private String[] adminGroups = {"adminGroup1"};
   private PolicyFile policyFile;
   private File policyFilePath;
+  final long NUM_PRIVS = 60;  // > SentryStore.PrivCleaner.NOTIFY_THRESHOLD
 
   @Before
   public void setup() throws Exception {
@@ -239,6 +240,107 @@ public class TestSentryStore {
     assertFalse(mPrivilege.getGrantOption());
   }
 
+  private void verifyOrphanCleanup() throws Exception {
+    boolean success = false;
+    int iterations = 30;
+    while (!success && iterations > 0) {
+      Thread.sleep(1000);
+      long numDBPrivs = sentryStore.countMSentryPrivileges();
+      if (numDBPrivs < NUM_PRIVS) {
+        assertEquals(0, numDBPrivs);
+        success = true;
+      }
+      iterations--;
+    }
+    assertTrue("Failed to cleanup orphaned privileges", success);
+  }
+
+  /**
+   * Create several privileges in the database, then delete the role that
+   * created them.  This makes them all orphans.  Wait a bit to ensure the
+   * cleanup thread runs, and expect them all to be gone from the database.
+   * @throws Exception
+   */
+  @Test
+  public void testPrivilegeCleanup() throws Exception {
+    final String roleName = "test-priv-cleanup";
+    final String grantor = "g1";
+    final String server = "server";
+    final String dBase = "db";
+    final String table = "table-";
+
+    sentryStore.createSentryRole(roleName);
+
+    // Create NUM_PRIVS unique privilege objects in the database
+    for (int i = 0; i < NUM_PRIVS; i++) {
+      TSentryPrivilege priv = new TSentryPrivilege();
+      priv.setPrivilegeScope("TABLE");
+      priv.setServerName(server);
+      priv.setAction(AccessConstants.ALL);
+      priv.setCreateTime(System.currentTimeMillis());
+      priv.setTableName(table + i);
+      priv.setDbName(dBase);
+      sentryStore.alterSentryRoleGrantPrivilege(grantor, roleName, priv);
+    }
+
+    // Make sure we really have the expected number of privs in the database
+    assertEquals(sentryStore.countMSentryPrivileges(), NUM_PRIVS);
+
+    // Now to make a bunch of orphans, we just remove the role that
+    // created them.
+    sentryStore.dropSentryRole(roleName);
+
+    // Now wait and see if the orphans get cleaned up
+    verifyOrphanCleanup();
+  }
+
+  /**
+   * Much like testPrivilegeCleanup, make a lot of privileges and make sure
+   * they get cleaned up.  The difference here is that the privileges are
+   * created by granting ALL and then removing SELECT - thus leaving INSERT.
+   * This test exists because the revocation plays havoc with the orphan
+   * cleanup thread.
+   * @throws Exception
+   */
+  @Test
+  public void testPrivilegeCleanup2() throws Exception {
+    final String roleName = "test-priv-cleanup";
+    final String grantor = "g1";
+    final String server = "server";
+    final String dBase = "db";
+    final String table = "table-";
+
+    sentryStore.createSentryRole(roleName);
+
+    // Create NUM_PRIVS unique privilege objects in the database once more,
+    // this time granting ALL and revoking SELECT to make INSERT.
+    for (int i=0 ; i < NUM_PRIVS; i++) {
+      TSentryPrivilege priv = new TSentryPrivilege();
+      priv.setPrivilegeScope("DATABASE");
+      priv.setServerName(server);
+      priv.setAction(AccessConstants.ALL);
+      priv.setCreateTime(System.currentTimeMillis());
+      priv.setTableName(table + i);
+      priv.setDbName(dBase);
+      priv.setGrantOption(TSentryGrantOption.TRUE);
+      sentryStore.alterSentryRoleGrantPrivilege(grantor, roleName, priv);
+
+      priv.setAction(AccessConstants.SELECT);
+      priv.setGrantOption(TSentryGrantOption.UNSET);
+      sentryStore.alterSentryRoleRevokePrivilege(grantor, roleName, priv);
+      // after having ALL and revoking SELECT, we should have INSERT
+      MSentryRole role = sentryStore.getMSentryRoleByName(roleName);
+      Set<MSentryPrivilege> privileges = role.getPrivileges();
+      assertEquals(privileges.toString(), i+1, privileges.size());
+      MSentryPrivilege mPrivilege = Iterables.get(privileges, 0);
+      assertEquals(AccessConstants.INSERT, mPrivilege.getAction());
+    }
+
+    // Drop the role and clean up as before
+    sentryStore.dropSentryRole(roleName);
+    verifyOrphanCleanup();
+  }
+
   @Test
   public void testGrantRevokeMultiPrivileges() throws Exception {
     String roleName = "test-privilege";