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/13 08:43:46 UTC

[9/9] incubator-sentry git commit: SENTRY-432: Synchronization of HDFS permissions to Sentry permissions (Arun Suresh via Lenni Kuff)

SENTRY-432: Synchronization of HDFS permissions to Sentry permissions (Arun Suresh via Lenni Kuff)

This change adds support for synchronizing HDFS permissions with permissions stored in
Sentry. This makes it easy to share data across components (Hive/Impala, MR, Spark, etc)
while managing all privileges in a centralized location - Sentry. This is done using new
plugins to the HMS, HDFS, and Sentry. The HMS plugin pushes table/partition path information
to the Sentry Service, the Sentry Service forwards the path information and all privilege
updates, to the HDFS NameNode plugin, which caches this information and updates the ACLs
accordingly.

The mapping of Sentry privileges to HDFS privileges is:
ALL -> Read/Write access to data files
SELECT -> Read access to data files
INSERT -> Write access to data files


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

Branch: refs/heads/master
Commit: 2e509e4bc4f06e1c207d5702aad55b290ef390a4
Parents: 49e6086
Author: Lenni Kuff <ls...@cloudera.com>
Authored: Wed Nov 12 22:58:53 2014 -0800
Committer: Lenni Kuff <ls...@cloudera.com>
Committed: Wed Nov 12 23:32:32 2014 -0800

----------------------------------------------------------------------
 bin/sentry                                      |    3 +
 pom.xml                                         |   61 +-
 .../SentryHiveAuthorizationTaskFactoryImpl.java |   14 +
 .../sentry/binding/hive/conf/HiveAuthzConf.java |    4 +-
 .../SentryMetastorePostEventListener.java       |  159 +-
 sentry-dist/pom.xml                             |    1 +
 sentry-dist/src/main/assembly/bin.xml           |   24 +-
 sentry-hdfs/pom.xml                             |   38 +
 sentry-hdfs/sentry-hdfs-common/.gitignore       |   18 +
 sentry-hdfs/sentry-hdfs-common/pom.xml          |  148 +
 .../hdfs/service/thrift/SentryHDFSService.java  | 3483 ++++++++++++++++++
 .../service/thrift/TAuthzUpdateResponse.java    |  603 +++
 .../hdfs/service/thrift/TPathChanges.java       |  765 ++++
 .../sentry/hdfs/service/thrift/TPathEntry.java  |  747 ++++
 .../sentry/hdfs/service/thrift/TPathsDump.java  |  549 +++
 .../hdfs/service/thrift/TPathsUpdate.java       |  748 ++++
 .../hdfs/service/thrift/TPermissionsUpdate.java |  810 ++++
 .../hdfs/service/thrift/TPrivilegeChanges.java  |  713 ++++
 .../hdfs/service/thrift/TRoleChanges.java       |  691 ++++
 .../java/org/apache/sentry/hdfs/AuthzPaths.java |   58 +
 .../apache/sentry/hdfs/AuthzPathsDumper.java    |   28 +
 .../apache/sentry/hdfs/AuthzPermissions.java    |   28 +
 .../java/org/apache/sentry/hdfs/HMSPaths.java   |  510 +++
 .../org/apache/sentry/hdfs/HMSPathsDumper.java  |  126 +
 .../org/apache/sentry/hdfs/MetastoreClient.java |   38 +
 .../org/apache/sentry/hdfs/PathsUpdate.java     |   88 +
 .../apache/sentry/hdfs/PermissionsUpdate.java   |   94 +
 .../sentry/hdfs/SentryHDFSServiceClient.java    |  229 ++
 .../apache/sentry/hdfs/ServiceConstants.java    |   70 +
 .../java/org/apache/sentry/hdfs/Updateable.java |   67 +
 .../sentry/hdfs/UpdateableAuthzPaths.java       |  153 +
 .../main/resources/sentry_hdfs_service.thrift   |   87 +
 .../org/apache/sentry/hdfs/TestHMSPaths.java    |  357 ++
 .../sentry/hdfs/TestHMSPathsFullDump.java       |  112 +
 .../sentry/hdfs/TestUpdateableAuthzPaths.java   |  156 +
 .../src/test/resources/hdfs-sentry.xml          |   22 +
 sentry-hdfs/sentry-hdfs-dist/pom.xml            |   79 +
 .../sentry-hdfs-namenode-plugin/.gitignore      |   18 +
 sentry-hdfs/sentry-hdfs-namenode-plugin/pom.xml |   63 +
 .../hdfs/SentryAuthorizationConstants.java      |   55 +
 .../sentry/hdfs/SentryAuthorizationInfo.java    |  237 ++
 .../hdfs/SentryAuthorizationProvider.java       |  372 ++
 .../apache/sentry/hdfs/SentryPermissions.java   |  220 ++
 .../org/apache/sentry/hdfs/SentryUpdater.java   |   61 +
 .../sentry/hdfs/UpdateableAuthzPermissions.java |  230 ++
 .../hdfs/MockSentryAuthorizationProvider.java   |   26 +
 .../sentry/hdfs/SentryAuthorizationInfoX.java   |   85 +
 .../hdfs/TestSentryAuthorizationProvider.java   |  164 +
 .../src/test/resources/hdfs-sentry.xml          |   33 +
 sentry-hdfs/sentry-hdfs-service/.gitignore      |   18 +
 sentry-hdfs/sentry-hdfs-service/pom.xml         |  104 +
 .../sentry/hdfs/ExtendedMetastoreClient.java    |  108 +
 .../org/apache/sentry/hdfs/MetastorePlugin.java |  257 ++
 .../sentry/hdfs/SentryHDFSServiceProcessor.java |  118 +
 .../hdfs/SentryHDFSServiceProcessorFactory.java |  108 +
 .../org/apache/sentry/hdfs/SentryPlugin.java    |  247 ++
 .../org/apache/sentry/hdfs/UpdateForwarder.java |  292 ++
 .../sentry/hdfs/UpdateablePermissions.java      |   62 +
 .../apache/sentry/hdfs/TestUpdateForwarder.java |  307 ++
 sentry-provider/sentry-provider-db/pom.xml      |   78 +-
 .../db/SentryMetastoreListenerPlugin.java       |   48 +
 .../provider/db/SentryPolicyStorePlugin.java    |   60 +
 .../provider/db/SimpleDBProviderBackend.java    |   40 +-
 .../db/service/persistent/SentryStore.java      |   81 +-
 .../thrift/SentryPolicyStoreProcessor.java      |   72 +-
 .../sentry/service/thrift/SentryService.java    |    1 +
 .../sentry/service/thrift/ServiceConstants.java |    7 +
 .../thrift/TestSentryPolicyStoreProcessor.java  |    1 +
 .../thrift/TestSentryServerWithoutKerberos.java |    4 +-
 sentry-tests/sentry-tests-hive/pom.xml          |   17 +
 .../tests/e2e/hdfs/TestHDFSIntegration.java     |  787 ++++
 .../sentry/tests/e2e/hive/StaticUserGroup.java  |    2 +
 ...actMetastoreTestWithStaticConfiguration.java |    2 +
 .../e2e/metastore/TestMetastoreEndToEnd.java    |   49 +-
 74 files changed, 16202 insertions(+), 113 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/2e509e4b/bin/sentry
----------------------------------------------------------------------
diff --git a/bin/sentry b/bin/sentry
index 0b98049..93809ea 100755
--- a/bin/sentry
+++ b/bin/sentry
@@ -72,6 +72,9 @@ then
   for f in ${SENTRY_HOME}/lib/server/*.jar; do
     HADOOP_CLASSPATH=${HADOOP_CLASSPATH}:${f}
   done
+  for f in ${SENTRY_HOME}/lib/plugins/*.jar; do
+    HADOOP_CLASSPATH=${HADOOP_CLASSPATH}:${f}
+  done
   exec $HADOOP jar ${SENTRY_HOME}/lib/${_CMD_JAR} org.apache.sentry.SentryMain ${args[@]}
 else
   exec ${SENTRY_HOME}/bin/config_tool ${args[@]}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/2e509e4b/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 8c59ba9..b0cdd9a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -68,7 +68,7 @@ limitations under the License.
     <derby.version>10.10.2.0</derby.version>
     <commons-cli.version>1.2</commons-cli.version>
     <hive.version>0.13.1-cdh5.3.0-SNAPSHOT</hive.version>
-    <hadoop.version>2.3.0-cdh5.1.0-SNAPSHOT</hadoop.version>
+    <hadoop.version>2.5.0-cdh5.2.0-SNAPSHOT</hadoop.version>
     <fest.reflect.version>1.4.1</fest.reflect.version>
     <guava.version>11.0.2</guava.version>
     <junit.version>4.9</junit.version>
@@ -151,6 +151,12 @@ limitations under the License.
       </dependency>
       <dependency>
         <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-hdfs</artifactId>
+        <version>${hadoop.version}</version>
+        <type>test-jar</type>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
         <artifactId>hadoop-minicluster</artifactId>
         <version>${hadoop.version}</version>
       </dependency>
@@ -336,6 +342,26 @@ limitations under the License.
       </dependency>
       <dependency>
         <groupId>org.apache.sentry</groupId>
+        <artifactId>sentry-hdfs-common</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.sentry</groupId>
+        <artifactId>sentry-hdfs-service</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.sentry</groupId>
+        <artifactId>sentry-hdfs-namenode-plugin</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.sentry</groupId>
+        <artifactId>sentry-hdfs-dist</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.sentry</groupId>
         <artifactId>sentry-provider-cache</artifactId>
         <version>${project.version}</version>
       </dependency>
@@ -403,6 +429,7 @@ limitations under the License.
     <module>sentry-provider</module>
     <module>sentry-policy</module>
     <module>sentry-tests</module>
+    <module>sentry-hdfs</module>
     <module>sentry-dist</module>
   </modules>
 
@@ -427,8 +454,38 @@ limitations under the License.
           <downloadSources>true</downloadSources>
           <workspaceActiveCodeStyleProfileName>GoogleStyle</workspaceActiveCodeStyleProfileName>
           <workspaceCodeStylesURL>https://google-styleguide.googlecode.com/svn/trunk/eclipse-java-google-style.xml</workspaceCodeStylesURL>
+          <sourceIncludes><include>src/gen/thrift/gen-javabean/**</include></sourceIncludes>
         </configuration>
       </plugin>
+
+<!---
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-shade-plugin</artifactId>
+        <version>2.1</version>
+        <executions>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>shade</goal>
+            </goals>
+            <configuration>
+              <artifactSet>
+                <includes>
+                  <include>org.apache.thrift:libthrift</include>
+                </includes>
+              </artifactSet>
+              <relocations>
+                <relocation>
+                  <pattern>org.apache.thrift</pattern>
+                  <shadedPattern>sentry.org.apache.thrift</shadedPattern>
+                </relocation>
+              </relocations>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+-->
     </plugins>
 
     <pluginManagement>
@@ -515,10 +572,12 @@ limitations under the License.
                   <exclude>**/.metadata/</exclude>
                   <!-- Maven working directory -->
                   <exclude>**/target/</exclude>
+                  <exclude>**/assembly/</exclude>
                   <!-- Pre commit testing generated files -->
                   <exclude>maven-repo/</exclude>
                   <exclude>test-output/</exclude>
                   <!-- Derby files which are created after test run -->
+                  <exclude>**/dependency-reduced-pom.xml</exclude>
                   <exclude>**/derby.log</exclude>
                   <exclude>**/service.properties</exclude>
                   <exclude>**/*.lck</exclude>

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/2e509e4b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java
index f38ee91..39a22c6 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java
@@ -50,11 +50,14 @@ import org.apache.hadoop.hive.ql.security.authorization.Privilege;
 import org.apache.hadoop.hive.ql.security.authorization.PrivilegeRegistry;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.sentry.core.model.db.AccessConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
 
 public class SentryHiveAuthorizationTaskFactoryImpl implements HiveAuthorizationTaskFactory {
 
+  private static final Logger LOG = LoggerFactory.getLogger(SentryHiveAuthorizationTaskFactoryImpl.class);
 
   public SentryHiveAuthorizationTaskFactoryImpl(HiveConf conf, Hive db) {
 
@@ -238,6 +241,7 @@ public class SentryHiveAuthorizationTaskFactoryImpl implements HiveAuthorization
       HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs) throws SemanticException {
     List<PrincipalDesc> principalDesc = analyzePrincipalListDef(
         (ASTNode) ast.getChild(0));
+    
     List<String> roles = new ArrayList<String>();
     for (int i = 1; i < ast.getChildCount(); i++) {
       roles.add(BaseSemanticAnalyzer.unescapeIdentifier(ast.getChild(i).getText()));
@@ -314,18 +318,28 @@ public class SentryHiveAuthorizationTaskFactoryImpl implements HiveAuthorization
       ASTNode child = (ASTNode) node.getChild(i);
       PrincipalType type = null;
       switch (child.getType()) {
+      case 880:
+        type = PrincipalType.USER;
+        break;
       case HiveParser.TOK_USER:
         type = PrincipalType.USER;
         break;
+      case 685:
+        type = PrincipalType.GROUP;
+        break;
       case HiveParser.TOK_GROUP:
         type = PrincipalType.GROUP;
         break;
+      case 782:
+        type = PrincipalType.ROLE;
+        break;
       case HiveParser.TOK_ROLE:
         type = PrincipalType.ROLE;
         break;
       }
       String principalName = BaseSemanticAnalyzer.unescapeIdentifier(child.getChild(0).getText());
       PrincipalDesc principalDesc = new PrincipalDesc(principalName, type);
+      LOG.debug("## Principal : [ " + principalName + ", " + type + "]");
       principalList.add(principalDesc);
     }
     return principalList;

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/2e509e4b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java
index 4d2a625..93f19f3 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java
@@ -18,7 +18,7 @@ package org.apache.sentry.binding.hive.conf;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.mortbay.log.Log;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -203,7 +203,7 @@ public class HiveAuthzConf extends Configuration {
       if (retVal == null) {
         retVal = AuthzConfVars.getDefault(varName);
       } else {
-        Log.warn("Using the deprecated config setting " + currentToDeprecatedProps.get(varName).getVar() +
+        LOG.warn("Using the deprecated config setting " + currentToDeprecatedProps.get(varName).getVar() +
             " instead of " + varName);
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/2e509e4b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/SentryMetastorePostEventListener.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/SentryMetastorePostEventListener.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/SentryMetastorePostEventListener.java
index 38bf8b2..3760fe9 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/SentryMetastorePostEventListener.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/SentryMetastorePostEventListener.java
@@ -25,10 +25,14 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.MetaStoreEventListener;
 import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.AlterTableEvent;
 import org.apache.hadoop.hive.metastore.events.CreateDatabaseEvent;
 import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
 import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.DropPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.DropTableEvent;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.sentry.SentryUserException;
@@ -38,24 +42,60 @@ import org.apache.sentry.core.common.Authorizable;
 import org.apache.sentry.core.model.db.Database;
 import org.apache.sentry.core.model.db.Server;
 import org.apache.sentry.core.model.db.Table;
+import org.apache.sentry.provider.db.SentryMetastoreListenerPlugin;
 import org.apache.sentry.provider.db.service.thrift.SentryPolicyServiceClient;
 import org.apache.sentry.service.thrift.SentryServiceClientFactory;
+import org.apache.sentry.service.thrift.ServiceConstants.ConfUtilties;
+import org.apache.sentry.service.thrift.ServiceConstants.ServerConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class SentryMetastorePostEventListener extends MetaStoreEventListener {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(SentryMetastoreListenerPlugin.class);
   private final SentryServiceClientFactory sentryClientFactory;
   private final HiveAuthzConf authzConf;
   private final Server server;
 
+  private List<SentryMetastoreListenerPlugin> sentryPlugins = new ArrayList<SentryMetastoreListenerPlugin>(); 
+
   public SentryMetastorePostEventListener(Configuration config) {
     super(config);
     sentryClientFactory = new SentryServiceClientFactory();
 
-    authzConf = HiveAuthzConf.getAuthzConf(new HiveConf());
+    authzConf = HiveAuthzConf.getAuthzConf((HiveConf)config);
     server = new Server(authzConf.get(AuthzConfVars.AUTHZ_SERVER_NAME.getVar()));
+    Iterable<String> pluginClasses = ConfUtilties.CLASS_SPLITTER
+        .split(config.get(ServerConfig.SENTRY_METASTORE_PLUGINS,
+            ServerConfig.SENTRY_METASTORE_PLUGINS_DEFAULT).trim());
+    try {
+      for (String pluginClassStr : pluginClasses) {
+        Class<?> clazz = config.getClassByName(pluginClassStr);
+        if (!SentryMetastoreListenerPlugin.class.isAssignableFrom(clazz)) {
+          throw new IllegalArgumentException("Class ["
+              + pluginClassStr + "] is not a "
+              + SentryMetastoreListenerPlugin.class.getName());
+        }
+        SentryMetastoreListenerPlugin plugin = (SentryMetastoreListenerPlugin) clazz
+            .getConstructor(Configuration.class).newInstance(config);
+        sentryPlugins.add(plugin);
+      }
+    } catch (Exception e) {
+      LOGGER.error("Could not initialize Plugin !!", e);
+      throw new RuntimeException(e);
+    }
   }
 
   @Override
   public void onCreateTable (CreateTableEvent tableEvent) throws MetaException {
+    if (tableEvent.getTable().getSd().getLocation() != null) {
+      String authzObj = tableEvent.getTable().getDbName() + "."
+          + tableEvent.getTable().getTableName();
+      String path = tableEvent.getTable().getSd().getLocation();
+      for (SentryMetastoreListenerPlugin plugin : sentryPlugins) {
+        plugin.addPath(authzObj, path);
+      }
+    }
     // drop the privileges on the given table, in case if anything was left
     // behind during the drop
     if (!syncWithPolicyStore(AuthzConfVars.AUTHZ_SYNC_CREATE_WITH_POLICY_STORE)) {
@@ -71,6 +111,13 @@ public class SentryMetastorePostEventListener extends MetaStoreEventListener {
 
   @Override
   public void onDropTable(DropTableEvent tableEvent) throws MetaException {
+    if (tableEvent.getTable().getSd().getLocation() != null) {
+      String authzObj = tableEvent.getTable().getDbName() + "."
+          + tableEvent.getTable().getTableName();
+      for (SentryMetastoreListenerPlugin plugin : sentryPlugins) {
+        plugin.removeAllPaths(authzObj, null);
+      }
+    }
     // drop the privileges on the given table
     if (!syncWithPolicyStore(AuthzConfVars.AUTHZ_SYNC_DROP_WITH_POLICY_STORE)) {
       return;
@@ -86,6 +133,13 @@ public class SentryMetastorePostEventListener extends MetaStoreEventListener {
   @Override
   public void onCreateDatabase(CreateDatabaseEvent dbEvent)
       throws MetaException {
+    if (dbEvent.getDatabase().getLocationUri() != null) {
+      String authzObj = dbEvent.getDatabase().getName();
+      String path = dbEvent.getDatabase().getLocationUri();
+      for (SentryMetastoreListenerPlugin plugin : sentryPlugins) {
+        plugin.addPath(authzObj, path);
+      }
+    }
     // drop the privileges on the database, incase anything left behind during
     // last drop db
     if (!syncWithPolicyStore(AuthzConfVars.AUTHZ_SYNC_CREATE_WITH_POLICY_STORE)) {
@@ -105,6 +159,12 @@ public class SentryMetastorePostEventListener extends MetaStoreEventListener {
    */
   @Override
   public void onDropDatabase(DropDatabaseEvent dbEvent) throws MetaException {
+    String authzObj = dbEvent.getDatabase().getName();
+    for (SentryMetastoreListenerPlugin plugin : sentryPlugins) {
+      List<String> tNames = dbEvent.getHandler().get_all_tables(authzObj);
+      plugin.removeAllPaths(authzObj, tNames);
+    }
+    dropSentryDbPrivileges(dbEvent.getDatabase().getName());
     if (!syncWithPolicyStore(AuthzConfVars.AUTHZ_SYNC_DROP_WITH_POLICY_STORE)) {
       return;
     }
@@ -121,9 +181,6 @@ public class SentryMetastorePostEventListener extends MetaStoreEventListener {
   @Override
   public void onAlterTable (AlterTableEvent tableEvent) throws MetaException {
     String oldTableName = null, newTableName = null;
-    if (!syncWithPolicyStore(AuthzConfVars.AUTHZ_SYNC_ALTER_WITH_POLICY_STORE)) {
-      return;
-    }
     // don't sync privileges if the operation has failed
     if (!tableEvent.getStatus()) {
       return;
@@ -135,10 +192,63 @@ public class SentryMetastorePostEventListener extends MetaStoreEventListener {
     if (tableEvent.getNewTable() != null) {
       newTableName = tableEvent.getNewTable().getTableName();
     }
-    if (!oldTableName.equalsIgnoreCase(newTableName)) {
-      renameSentryTablePrivilege(tableEvent.getOldTable().getDbName(),
-          oldTableName, tableEvent.getNewTable().getDbName(), newTableName);
+    renameSentryTablePrivilege(tableEvent.getOldTable().getDbName(),
+        oldTableName, tableEvent.getOldTable().getSd().getLocation(),
+        tableEvent.getNewTable().getDbName(), newTableName,
+        tableEvent.getNewTable().getSd().getLocation());
+  }
+
+  @Override
+  public void onAlterPartition(AlterPartitionEvent partitionEvent)
+      throws MetaException {
+    // don't sync privileges if the operation has failed
+    if (!partitionEvent.getStatus()) {
+      return;
+    }
+    String oldLoc = null, newLoc = null;
+    if (partitionEvent.getOldPartition() != null) {
+      oldLoc = partitionEvent.getOldPartition().getSd().getLocation();
+    }
+    if (partitionEvent.getNewPartition() != null) {
+      newLoc = partitionEvent.getNewPartition().getSd().getLocation();
     }
+
+    if ((oldLoc != null) && (newLoc != null) && (!oldLoc.equals(newLoc))) {
+      String authzObj =
+          partitionEvent.getOldPartition().getDbName() + "."
+              + partitionEvent.getOldPartition().getTableName();
+      for (SentryMetastoreListenerPlugin plugin : sentryPlugins) {
+        plugin.renameAuthzObject(authzObj, oldLoc,
+            authzObj, newLoc);
+      }
+    }
+  }
+
+  @Override
+  public void onAddPartition(AddPartitionEvent partitionEvent)
+      throws MetaException {
+    for (Partition part : partitionEvent.getPartitions()) {
+      if ((part.getSd() != null) && (part.getSd().getLocation() != null)) {
+        String authzObj = part.getDbName() + "." + part.getTableName();
+        String path = part.getSd().getLocation();
+        for (SentryMetastoreListenerPlugin plugin : sentryPlugins) {
+          plugin.addPath(authzObj, path);
+        }
+      }
+    }
+    super.onAddPartition(partitionEvent);
+  }
+
+  @Override
+  public void onDropPartition(DropPartitionEvent partitionEvent)
+      throws MetaException {
+    String authzObj = partitionEvent.getTable().getDbName() + "."
+        + partitionEvent.getTable().getTableName();
+    String path = partitionEvent.getPartition().getSd().getLocation();
+    for (SentryMetastoreListenerPlugin plugin : sentryPlugins) {
+      plugin.removePath(authzObj, path);
+    }
+    super.onDropPartition(partitionEvent);
   }
 
   private SentryPolicyServiceClient getSentryServiceClient()
@@ -194,7 +304,7 @@ public class SentryMetastorePostEventListener extends MetaStoreEventListener {
   }
 
   private void renameSentryTablePrivilege(String oldDbName, String oldTabName,
-      String newDbName, String newTabName)
+      String oldPath, String newDbName, String newTabName, String newPath)
       throws MetaException {
     List<Authorizable> oldAuthorizableTable = new ArrayList<Authorizable>();
     oldAuthorizableTable.add(server);
@@ -206,18 +316,26 @@ public class SentryMetastorePostEventListener extends MetaStoreEventListener {
     newAuthorizableTable.add(new Database(newDbName));
     newAuthorizableTable.add(new Table(newTabName));
 
-    try {
-      String requestorUserName = UserGroupInformation.getCurrentUser()
-          .getShortUserName();
-      SentryPolicyServiceClient sentryClient = getSentryServiceClient();
-      sentryClient.renamePrivileges(requestorUserName, oldAuthorizableTable, newAuthorizableTable);
-    } catch (SentryUserException e) {
-      throw new MetaException(
-          "Failed to remove Sentry policies for rename table " + oldDbName
-              + "." + oldTabName + "to " + newDbName + "." + newTabName
-              + " Error: " + e.getMessage());
-    } catch (IOException e) {
-      throw new MetaException("Failed to find local user " + e.getMessage());
+    if (!oldTabName.equalsIgnoreCase(newTabName)
+        && syncWithPolicyStore(AuthzConfVars.AUTHZ_SYNC_ALTER_WITH_POLICY_STORE)) {
+      try {
+        String requestorUserName = UserGroupInformation.getCurrentUser()
+            .getShortUserName();
+        SentryPolicyServiceClient sentryClient = getSentryServiceClient();
+        sentryClient.renamePrivileges(requestorUserName, oldAuthorizableTable, newAuthorizableTable);
+      } catch (SentryUserException e) {
+        throw new MetaException(
+            "Failed to remove Sentry policies for rename table " + oldDbName
+            + "." + oldTabName + "to " + newDbName + "." + newTabName
+            + " Error: " + e.getMessage());
+      } catch (IOException e) {
+        throw new MetaException("Failed to find local user " + e.getMessage());
+      }
+    }
+    // The HDFS plugin needs to know if it's a path change (set location)
+    for (SentryMetastoreListenerPlugin plugin : sentryPlugins) {
+      plugin.renameAuthzObject(oldDbName + "." + oldTabName, oldPath,
+          newDbName + "." + newTabName, newPath);
     }
   }
 
@@ -225,4 +343,5 @@ public class SentryMetastorePostEventListener extends MetaStoreEventListener {
     return "true"
         .equalsIgnoreCase((authzConf.get(syncConfVar.getVar(), "true")));
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/2e509e4b/sentry-dist/pom.xml
----------------------------------------------------------------------
diff --git a/sentry-dist/pom.xml b/sentry-dist/pom.xml
index cd7126b..4eb1d9c 100644
--- a/sentry-dist/pom.xml
+++ b/sentry-dist/pom.xml
@@ -77,6 +77,7 @@ limitations under the License.
   </dependencies>
   <build>
     <plugins>
+
      <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-assembly-plugin</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/2e509e4b/sentry-dist/src/main/assembly/bin.xml
----------------------------------------------------------------------
diff --git a/sentry-dist/src/main/assembly/bin.xml b/sentry-dist/src/main/assembly/bin.xml
index 258e63c..f1d301b 100644
--- a/sentry-dist/src/main/assembly/bin.xml
+++ b/sentry-dist/src/main/assembly/bin.xml
@@ -46,6 +46,18 @@
         <exclude>org.apache.derby:derby</exclude>
       </excludes>
     </dependencySet>
+<!--
+    <dependencySet>
+      <outputDirectory>lib/plugins</outputDirectory>
+      <unpack>true</unpack>
+      <useTransitiveDependencies>false</useTransitiveDependencies>
+      <includes>
+        <include>org.apache.sentry:sentry-provider-db</include>
+        <include>org.apache.sentry:sentry-hdfs-common</include>
+        <include>org.apache.sentry:sentry-hdfs-namenode-plugin</include>
+      </includes>
+    </dependencySet>
+-->
     <dependencySet>
       <outputDirectory>lib/server</outputDirectory>
       <unpack>false</unpack>
@@ -57,7 +69,6 @@
         <include>com.jolbox:bonecp</include>
         <include>org.apache.hive:hive-beeline</include>
         <include>org.apache.derby:derby</include>
-        <include>org.apache.derby:derby</include>
       </includes>
     </dependencySet>
   </dependencySets>
@@ -80,6 +91,7 @@
         <exclude>sentry-provider/**</exclude>
         <exclude>sentry-policy/**</exclude>
         <exclude>sentry-tests/**</exclude>
+        <exclude>sentry-hdfs/**</exclude>
       </excludes>
 
       <includes>
@@ -95,6 +107,16 @@
       <outputDirectory>/</outputDirectory>
     </fileSet>
     <fileSet>
+      <directory>${project.parent.basedir}/sentry-hdfs/sentry-hdfs-dist/target</directory>
+      <includes>
+        <include>sentry-hdfs-*.jar</include>
+      </includes>
+      <excludes>
+        <exclude>sentry-hdfs-dist-*.jar</exclude>
+      </excludes>
+      <outputDirectory>lib/plugins</outputDirectory>
+    </fileSet>
+    <fileSet>
       <directory>${project.parent.basedir}/sentry-provider/sentry-provider-db/src/main/resources</directory>
       <includes>
         <include>**/*</include>

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/2e509e4b/sentry-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/sentry-hdfs/pom.xml b/sentry-hdfs/pom.xml
new file mode 100644
index 0000000..1455235
--- /dev/null
+++ b/sentry-hdfs/pom.xml
@@ -0,0 +1,38 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.sentry</groupId>
+    <artifactId>sentry</artifactId>
+    <version>1.5.0-incubating-SNAPSHOT</version>
+  </parent>
+
+  <artifactId>sentry-hdfs</artifactId>
+  <name>Sentry HDFS</name>
+  <packaging>pom</packaging>
+  <modules>
+    <module>sentry-hdfs-common</module>
+    <module>sentry-hdfs-service</module>
+    <module>sentry-hdfs-namenode-plugin</module>
+    <module>sentry-hdfs-dist</module>
+  </modules>
+
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/2e509e4b/sentry-hdfs/sentry-hdfs-common/.gitignore
----------------------------------------------------------------------
diff --git a/sentry-hdfs/sentry-hdfs-common/.gitignore b/sentry-hdfs/sentry-hdfs-common/.gitignore
new file mode 100644
index 0000000..91ad75b
--- /dev/null
+++ b/sentry-hdfs/sentry-hdfs-common/.gitignore
@@ -0,0 +1,18 @@
+*.class
+target/
+.classpath
+.project
+.settings
+.metadata
+.idea/
+*.iml
+derby.log
+datanucleus.log
+sentry-core/sentry-core-common/src/gen
+**/TempStatsStore/
+# Package Files #
+*.jar
+*.war
+*.ear
+test-output/
+maven-repo/

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/2e509e4b/sentry-hdfs/sentry-hdfs-common/pom.xml
----------------------------------------------------------------------
diff --git a/sentry-hdfs/sentry-hdfs-common/pom.xml b/sentry-hdfs/sentry-hdfs-common/pom.xml
new file mode 100644
index 0000000..511bc53
--- /dev/null
+++ b/sentry-hdfs/sentry-hdfs-common/pom.xml
@@ -0,0 +1,148 @@
+<?xml version="1.0"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+<project xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd" xmlns="http://maven.apache.org/POM/4.0.0"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.sentry</groupId>
+    <artifactId>sentry-hdfs</artifactId>
+    <version>1.5.0-incubating-SNAPSHOT</version>
+  </parent>
+
+  <artifactId>sentry-hdfs-common</artifactId>
+  <name>Sentry HDFS Common</name>
+
+  <dependencies>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minicluster</artifactId>
+      <version>2.5.0</version>
+      <scope>provided</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
+      <artifactId>hive-metastore</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <version>2.5.0</version>
+      <scope>provided</scope>
+    </dependency>
+  </dependencies>
+  <build>
+    <sourceDirectory>${basedir}/src/main/java</sourceDirectory>
+    <testSourceDirectory>${basedir}/src/test/java</testSourceDirectory>
+    <plugins>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>add-source</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>add-source</goal>
+            </goals>
+            <configuration>
+              <sources>
+                <source>src/gen/thrift/gen-javabean</source>
+              </sources>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+  <profiles>
+    <profile>
+      <id>thriftif</id>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-antrun-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>generate-thrift-sources</id>
+                <phase>generate-sources</phase>
+                <configuration>
+                  <target>
+                    <taskdef name="for" classname="net.sf.antcontrib.logic.ForTask"
+                      classpathref="maven.plugin.classpath" />
+                    <property name="thrift.args" value="-I ${thrift.home} --gen java:beans,hashcode"/>
+                    <property name="thrift.gen.dir" value="${basedir}/src/gen/thrift"/>
+                    <delete dir="${thrift.gen.dir}"/>
+                    <mkdir dir="${thrift.gen.dir}"/>
+                    <for param="thrift.file">
+                      <path>
+                        <fileset dir="${basedir}/src/main/resources/" includes="**/*.thrift" />
+                      </path>
+                      <sequential>
+                        <echo message="Generating Thrift code for @{thrift.file}"/>
+                        <exec executable="${thrift.home}/bin/thrift"  failonerror="true" dir=".">
+                          <arg line="${thrift.args} -I ${basedir}/src/main/resources/ -o ${thrift.gen.dir} @{thrift.file} " />
+                        </exec>
+                      </sequential>
+                    </for>
+                  </target>
+                </configuration>
+                <goals>
+                  <goal>run</goal>
+                </goals>
+              </execution>
+            </executions>
+          </plugin>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-enforcer-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>enforce-property</id>
+                <goals>
+                  <goal>enforce</goal>
+                </goals>
+                <configuration>
+                  <rules>
+                    <requireProperty>
+                      <property>thrift.home</property>
+                    </requireProperty>
+                  </rules>
+                  <fail>true</fail>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+
+</project>