You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by sd...@apache.org on 2016/03/23 03:22:34 UTC

[3/5] sentry git commit: SENTRY-1138: Extract common classes for binding-hive-v1 and binding-hive-v2 (Dapeng Sun, reviewed by Colin Ma)

http://git-wip-us.apache.org/repos/asf/sentry/blob/7a30c819/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/MetastoreAuthzBindingBase.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/MetastoreAuthzBindingBase.java b/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/MetastoreAuthzBindingBase.java
new file mode 100644
index 0000000..fb7d246
--- /dev/null
+++ b/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/MetastoreAuthzBindingBase.java
@@ -0,0 +1,450 @@
+/**
+ * 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.binding.metastore;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+import javax.security.auth.login.LoginException;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.MetaStorePreEventListener;
+import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.events.PreAddPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.PreAlterPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.PreAlterTableEvent;
+import org.apache.hadoop.hive.metastore.events.PreCreateTableEvent;
+import org.apache.hadoop.hive.metastore.events.PreDropDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.PreDropPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.PreDropTableEvent;
+import org.apache.hadoop.hive.metastore.events.PreEventContext;
+import org.apache.hadoop.hive.ql.plan.HiveOperation;
+import org.apache.hadoop.hive.shims.Utils;
+import org.apache.sentry.binding.hive.authz.HiveAuthzBinding;
+import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
+import org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars;
+import org.apache.sentry.core.common.utils.PathUtils;
+import org.apache.sentry.core.model.db.AccessURI;
+import org.apache.sentry.core.model.db.DBModelAuthorizable;
+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 com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+
+/**
+ * Sentry binding for Hive Metastore. The binding is integrated into Metastore
+ * via the pre-event listener which are fired prior to executing the metadata
+ * action. This point we are only authorizing metadata writes since the listners
+ * are not fired from read events. Each action builds a input and output
+ * hierarchy as per the objects used in the given operations. This is then
+ * passed down to the hive binding which handles the authorization. This ensures
+ * that we follow the same privilege model and policies.
+ */
+public abstract class MetastoreAuthzBindingBase extends MetaStorePreEventListener {
+
+  /**
+   * Build the set of object hierarchies ie fully qualified db model objects
+   */
+  protected static class HierarcyBuilder {
+    private List<List<DBModelAuthorizable>> authHierarchy;
+
+    public HierarcyBuilder() {
+      authHierarchy = new ArrayList<List<DBModelAuthorizable>>();
+    }
+
+    public HierarcyBuilder addServerToOutput(Server server) {
+      List<DBModelAuthorizable> serverHierarchy = new ArrayList<DBModelAuthorizable>();
+      serverHierarchy.add(server);
+      authHierarchy.add(serverHierarchy);
+      return this;
+    }
+
+    public HierarcyBuilder addDbToOutput(Server server, String dbName) {
+      List<DBModelAuthorizable> dbHierarchy = new ArrayList<DBModelAuthorizable>();
+      addServerToOutput(server);
+      dbHierarchy.add(server);
+      dbHierarchy.add(new Database(dbName));
+      authHierarchy.add(dbHierarchy);
+      return this;
+    }
+
+    public HierarcyBuilder addUriToOutput(Server server, String uriPath,
+        String warehouseDirPath) throws MetaException {
+      List<DBModelAuthorizable> uriHierarchy = new ArrayList<DBModelAuthorizable>();
+      addServerToOutput(server);
+      uriHierarchy.add(server);
+      try {
+        uriHierarchy.add(new AccessURI(PathUtils.parseDFSURI(warehouseDirPath,
+            uriPath)));
+      } catch (URISyntaxException e) {
+        throw new MetaException("Error paring the URI " + e.getMessage());
+      }
+      authHierarchy.add(uriHierarchy);
+      return this;
+    }
+
+    public HierarcyBuilder addTableToOutput(Server server, String dbName,
+        String tableName) {
+      List<DBModelAuthorizable> tableHierarchy = new ArrayList<DBModelAuthorizable>();
+      addDbToOutput(server, dbName);
+      tableHierarchy.add(server);
+      tableHierarchy.add(new Database(dbName));
+      tableHierarchy.add(new Table(tableName));
+      authHierarchy.add(tableHierarchy);
+      return this;
+    }
+
+    public List<List<DBModelAuthorizable>> build() {
+      return authHierarchy;
+    }
+  }
+
+  private HiveAuthzConf authzConf;
+  private final Server authServer;
+  private final HiveConf hiveConf;
+  private final ImmutableSet<String> serviceUsers;
+  private HiveAuthzBinding hiveAuthzBinding;
+  private final String warehouseDir;
+  protected static boolean sentryCacheOutOfSync = false;
+
+  public MetastoreAuthzBindingBase(Configuration config) throws Exception {
+    super(config);
+    String hiveAuthzConf = config.get(HiveAuthzConf.HIVE_SENTRY_CONF_URL);
+    if (hiveAuthzConf == null
+        || (hiveAuthzConf = hiveAuthzConf.trim()).isEmpty()) {
+      throw new IllegalArgumentException("Configuration key "
+          + HiveAuthzConf.HIVE_SENTRY_CONF_URL + " value '" + hiveAuthzConf
+          + "' is invalid.");
+    }
+    try {
+      authzConf = new HiveAuthzConf(new URL(hiveAuthzConf));
+    } catch (MalformedURLException e) {
+      throw new IllegalArgumentException("Configuration key "
+          + HiveAuthzConf.HIVE_SENTRY_CONF_URL + " specifies a malformed URL '"
+          + hiveAuthzConf + "'", e);
+    }
+    hiveConf = new HiveConf(config, this.getClass());
+    this.authServer = new Server(authzConf.get(AuthzConfVars.AUTHZ_SERVER_NAME
+        .getVar()));
+    serviceUsers = ImmutableSet.copyOf(toTrimedLower(Sets.newHashSet(authzConf
+        .getStrings(AuthzConfVars.AUTHZ_METASTORE_SERVICE_USERS.getVar(),
+            new String[] { "" }))));
+    warehouseDir = hiveConf.getVar(HiveConf.ConfVars.METASTOREWAREHOUSE);
+
+  }
+
+  /**
+   * Main listener callback which is the entry point for Sentry
+   */
+  @Override
+  public void onEvent(PreEventContext context) throws MetaException,
+      NoSuchObjectException, InvalidOperationException {
+
+    if (!needsAuthorization(getUserName())) {
+      return;
+    }
+    switch (context.getEventType()) {
+    case CREATE_TABLE:
+      authorizeCreateTable((PreCreateTableEvent) context);
+      break;
+    case DROP_TABLE:
+      authorizeDropTable((PreDropTableEvent) context);
+      break;
+    case ALTER_TABLE:
+      authorizeAlterTable((PreAlterTableEvent) context);
+      break;
+    case ADD_PARTITION:
+      authorizeAddPartition((PreAddPartitionEvent) context);
+      break;
+    case DROP_PARTITION:
+      authorizeDropPartition((PreDropPartitionEvent) context);
+      break;
+    case ALTER_PARTITION:
+      authorizeAlterPartition((PreAlterPartitionEvent) context);
+      break;
+    case CREATE_DATABASE:
+      authorizeCreateDatabase();
+      break;
+    case DROP_DATABASE:
+      authorizeDropDatabase((PreDropDatabaseEvent) context);
+      break;
+    case LOAD_PARTITION_DONE:
+      // noop for now
+      break;
+    default:
+      break;
+    }
+  }
+
+  private void authorizeCreateDatabase()
+      throws InvalidOperationException, MetaException {
+    authorizeMetastoreAccess(HiveOperation.CREATEDATABASE,
+        new HierarcyBuilder().addServerToOutput(getAuthServer()).build(),
+        new HierarcyBuilder().addServerToOutput(getAuthServer()).build());
+  }
+
+  private void authorizeDropDatabase(PreDropDatabaseEvent context)
+      throws InvalidOperationException, MetaException {
+    authorizeMetastoreAccess(HiveOperation.DROPDATABASE,
+ new HierarcyBuilder()
+.addDbToOutput(getAuthServer(),
+            context.getDatabase().getName()).build(),
+        new HierarcyBuilder().addDbToOutput(getAuthServer(),
+            context.getDatabase().getName()).build());
+  }
+
+  private void authorizeCreateTable(PreCreateTableEvent context)
+      throws InvalidOperationException, MetaException {
+    HierarcyBuilder inputBuilder = new HierarcyBuilder();
+    inputBuilder.addDbToOutput(getAuthServer(), context.getTable().getDbName());
+    HierarcyBuilder outputBuilder = new HierarcyBuilder();
+    outputBuilder.addDbToOutput(getAuthServer(), context.getTable().getDbName());
+
+    if (!StringUtils.isEmpty(context.getTable().getSd().getLocation())) {
+      String uriPath;
+      try {
+        uriPath = PathUtils.parseDFSURI(warehouseDir,
+            getSdLocation(context.getTable().getSd()));
+      } catch(URISyntaxException e) {
+        throw new MetaException(e.getMessage());
+      }
+      inputBuilder.addUriToOutput(getAuthServer(), uriPath, warehouseDir);
+    }
+    authorizeMetastoreAccess(HiveOperation.CREATETABLE, inputBuilder.build(),
+        outputBuilder.build());
+  }
+
+  private void authorizeDropTable(PreDropTableEvent context)
+      throws InvalidOperationException, MetaException {
+    authorizeMetastoreAccess(
+        HiveOperation.DROPTABLE,
+        new HierarcyBuilder().addTableToOutput(getAuthServer(),
+            context.getTable().getDbName(), context.getTable().getTableName())
+            .build(),
+        new HierarcyBuilder().addTableToOutput(getAuthServer(),
+            context.getTable().getDbName(), context.getTable().getTableName())
+            .build());
+  }
+
+  private void authorizeAlterTable(PreAlterTableEvent context)
+      throws InvalidOperationException, MetaException {
+    /*
+     * There are multiple alter table options and it's tricky to figure which is
+     * attempted here. Currently all alter table needs full level privilege
+     * except the for setting location which also needs a privile on URI. Hence
+     * we set initially set the operation to ALTERTABLE_ADDCOLS. If the client
+     * has specified the location, then change to ALTERTABLE_LOCATION
+     */
+    HiveOperation operation = HiveOperation.ALTERTABLE_ADDCOLS;
+    HierarcyBuilder inputBuilder = new HierarcyBuilder();
+    inputBuilder.addTableToOutput(getAuthServer(), context.getOldTable()
+        .getDbName(), context.getOldTable().getTableName());
+    HierarcyBuilder outputBuilder = new HierarcyBuilder();
+    outputBuilder.addTableToOutput(getAuthServer(), context.getOldTable()
+        .getDbName(), context.getOldTable().getTableName());
+
+    // if the operation requires location change, then add URI privilege check
+    String oldLocationUri;
+    String newLocationUri;
+    try {
+      oldLocationUri = PathUtils.parseDFSURI(warehouseDir,
+          getSdLocation(context.getOldTable().getSd()));
+      newLocationUri = PathUtils.parseDFSURI(warehouseDir,
+          getSdLocation(context.getNewTable().getSd()));
+    } catch (URISyntaxException e) {
+      throw new MetaException(e.getMessage());
+    }
+    if (oldLocationUri.compareTo(newLocationUri) != 0) {
+      outputBuilder.addUriToOutput(getAuthServer(), newLocationUri,
+          warehouseDir);
+      operation = HiveOperation.ALTERTABLE_LOCATION;
+    }
+    authorizeMetastoreAccess(
+        operation,
+        inputBuilder.build(), outputBuilder.build());
+
+  }
+
+  private void authorizeAddPartition(PreAddPartitionEvent context)
+      throws InvalidOperationException, MetaException, NoSuchObjectException {
+    for (Partition mapiPart : context.getPartitions()) {
+	    HierarcyBuilder inputBuilder = new HierarcyBuilder();
+      inputBuilder.addTableToOutput(getAuthServer(), mapiPart
+          .getDbName(), mapiPart.getTableName());
+      HierarcyBuilder outputBuilder = new HierarcyBuilder();
+	    outputBuilder.addTableToOutput(getAuthServer(), mapiPart
+	        .getDbName(), mapiPart.getTableName());
+	    // check if we need to validate URI permissions when storage location is
+	    // non-default, ie something not under the parent table
+
+      String partitionLocation = null;
+      if (mapiPart.isSetSd()) {
+        partitionLocation = mapiPart.getSd().getLocation();
+	    }
+	    if (!StringUtils.isEmpty(partitionLocation)) {
+	      String tableLocation = context
+	          .getHandler()
+	          .get_table(mapiPart.getDbName(),
+	              mapiPart.getTableName()).getSd().getLocation();
+	      String uriPath;
+	      try {
+	        uriPath = PathUtils.parseDFSURI(warehouseDir, mapiPart
+	            .getSd().getLocation());
+	      } catch (URISyntaxException e) {
+	        throw new MetaException(e.getMessage());
+	      }
+        if (!partitionLocation.equals(tableLocation) &&
+            !partitionLocation.startsWith(tableLocation + File.separator)) {
+          outputBuilder.addUriToOutput(getAuthServer(), uriPath, warehouseDir);
+	      }
+	    }
+      authorizeMetastoreAccess(HiveOperation.ALTERTABLE_ADDPARTS,
+	        inputBuilder.build(), outputBuilder.build());
+    }
+  }
+
+  protected void authorizeDropPartition(PreDropPartitionEvent context)
+      throws InvalidOperationException, MetaException {
+    authorizeMetastoreAccess(
+        HiveOperation.ALTERTABLE_DROPPARTS,
+        new HierarcyBuilder().addTableToOutput(getAuthServer(),
+            context.getPartition().getDbName(),
+            context.getPartition().getTableName()).build(),
+        new HierarcyBuilder().addTableToOutput(getAuthServer(),
+            context.getPartition().getDbName(),
+            context.getPartition().getTableName()).build());
+  }
+
+  private void authorizeAlterPartition(PreAlterPartitionEvent context)
+      throws InvalidOperationException, MetaException, NoSuchObjectException {
+    /*
+     * There are multiple alter partition options and it's tricky to figure out
+     * which is attempted here. Currently all alter partition need full level
+     * privilege except the for setting location which also needs a privilege on
+     * URI. Currently we don't try to distinguish the operation type. All alter
+     * partitions are treated as set-location
+     */
+    HierarcyBuilder inputBuilder = new HierarcyBuilder().addTableToOutput(
+        getAuthServer(), context.getDbName(), context.getTableName());
+    HierarcyBuilder outputBuilder = new HierarcyBuilder().addTableToOutput(
+        getAuthServer(), context.getDbName(), context.getTableName());
+
+    Partition partition = context.getNewPartition();
+    String partitionLocation = getSdLocation(partition.getSd());
+    if (!StringUtils.isEmpty(partitionLocation)) {
+      String tableLocation = context.getHandler().get_table(
+          partition.getDbName(), partition.getTableName()).getSd().getLocation();
+
+      String uriPath;
+      try {
+        uriPath = PathUtils.parseDFSURI(warehouseDir, partitionLocation);
+        } catch (URISyntaxException e) {
+        throw new MetaException(e.getMessage());
+      }
+      if (!partitionLocation.startsWith(tableLocation + File.separator)) {
+        outputBuilder.addUriToOutput(getAuthServer(), uriPath, warehouseDir);
+      }
+    }
+    authorizeMetastoreAccess(
+        HiveOperation.ALTERPARTITION_LOCATION,
+        inputBuilder.build(), outputBuilder.build());
+  }
+
+  protected InvalidOperationException invalidOperationException(Exception e) {
+    InvalidOperationException ex = new InvalidOperationException(e.getMessage());
+    ex.initCause(e.getCause());
+    return ex;
+  }
+
+  /**
+   * Assemble the required privileges and requested privileges. Validate using
+   * Hive bind auth provider
+   * @param hiveOp
+   * @param inputHierarchy
+   * @param outputHierarchy
+   * @throws InvalidOperationException
+   */
+  protected abstract void authorizeMetastoreAccess(HiveOperation hiveOp,
+      List<List<DBModelAuthorizable>> inputHierarchy,
+      List<List<DBModelAuthorizable>> outputHierarchy)
+      throws InvalidOperationException;
+
+  public Server getAuthServer() {
+    return authServer;
+  }
+
+  private boolean needsAuthorization(String userName) {
+    return !serviceUsers.contains(userName);
+  }
+
+  private static Set<String> toTrimedLower(Set<String> s) {
+    Set<String> result = Sets.newHashSet();
+    for (String v : s) {
+      result.add(v.trim().toLowerCase());
+    }
+    return result;
+  }
+
+  protected HiveAuthzBinding getHiveAuthzBinding() throws Exception {
+    if (hiveAuthzBinding == null) {
+      hiveAuthzBinding = new HiveAuthzBinding(HiveAuthzBinding.HiveHook.HiveMetaStore, hiveConf, authzConf);
+    }
+    return hiveAuthzBinding;
+  }
+
+  protected String getUserName() throws MetaException {
+    try {
+      return Utils.getUGI().getShortUserName();
+    } catch (LoginException e) {
+      throw new MetaException("Failed to get username " + e.getMessage());
+    } catch (IOException e) {
+      throw new MetaException("Failed to get username " + e.getMessage());
+    }
+  }
+
+  private String getSdLocation(StorageDescriptor sd) {
+    if (sd == null) {
+      return "";
+    } else {
+      return sd.getLocation();
+    }
+  }
+
+  public static boolean isSentryCacheOutOfSync() {
+    return sentryCacheOutOfSync;
+  }
+
+  public static void setSentryCacheOutOfSync(boolean sentryCacheOutOfSync) {
+    MetastoreAuthzBindingBase.sentryCacheOutOfSync = sentryCacheOutOfSync;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sentry/blob/7a30c819/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/SentryHiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/SentryHiveMetaStoreClient.java b/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/SentryHiveMetaStoreClient.java
new file mode 100644
index 0000000..b5df287
--- /dev/null
+++ b/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/SentryHiveMetaStoreClient.java
@@ -0,0 +1,161 @@
+/**
+ * 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.binding.metastore;
+
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.List;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaHookLoader;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.UnknownDBException;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.plan.HiveOperation;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.sentry.binding.hive.HiveAuthzBindingHookBase;
+import org.apache.sentry.binding.hive.authz.HiveAuthzBinding;
+import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
+import org.apache.thrift.TException;
+
+public class SentryHiveMetaStoreClient extends HiveMetaStoreClient implements
+    IMetaStoreClient {
+
+  private HiveAuthzBinding hiveAuthzBinding;
+  private HiveAuthzConf authzConf;
+
+  public SentryHiveMetaStoreClient(HiveConf conf) throws MetaException {
+    super(conf);
+  }
+
+  public SentryHiveMetaStoreClient(HiveConf conf, HiveMetaHookLoader hookLoader)
+      throws MetaException {
+    super(conf, hookLoader);
+  }
+
+  @Override
+  public List<String> getDatabases(String databasePattern) throws MetaException {
+    return filterDatabases(super.getDatabases(databasePattern));
+  }
+
+  @Override
+  public List<String> getAllDatabases() throws MetaException {
+    return filterDatabases(super.getAllDatabases());
+  }
+
+  @Override
+  public List<String> getTables(String dbName, String tablePattern)
+      throws MetaException {
+    return filterTables(dbName, super.getTables(dbName, tablePattern));
+  }
+
+  @Override
+  public List<String> getAllTables(String dbName) throws MetaException {
+    return filterTables(dbName, super.getAllTables(dbName));
+  }
+
+  @Override
+  public List<String> listTableNamesByFilter(String dbName, String filter,
+      short maxTables) throws InvalidOperationException, UnknownDBException,
+      TException {
+    return filterTables(dbName,
+        super.listTableNamesByFilter(dbName, filter, maxTables));
+  }
+
+  /**
+   * Invoke Hive database filtering that removes the entries which use has no
+   * privileges to access
+   *
+   * @param dbList
+   * @return
+   * @throws MetaException
+   */
+  private List<String> filterDatabases(List<String> dbList)
+      throws MetaException {
+    try {
+      return HiveAuthzBindingHookBase.filterShowDatabases(getHiveAuthzBinding(),
+          dbList, HiveOperation.SHOWDATABASES, getUserName());
+    } catch (SemanticException e) {
+      throw new MetaException("Error getting DB list " + e.getMessage());
+    }
+  }
+
+  /**
+   * Invoke Hive table filtering that removes the entries which use has no
+   * privileges to access
+   *
+   * @param dbList
+   * @return
+   * @throws MetaException
+   */
+  private List<String> filterTables(String dbName, List<String> tabList)
+      throws MetaException {
+    try {
+      return HiveAuthzBindingHookBase.filterShowTables(getHiveAuthzBinding(),
+          tabList, HiveOperation.SHOWTABLES, getUserName(), dbName);
+    } catch (SemanticException e) {
+      throw new MetaException("Error getting Table list " + e.getMessage());
+    }
+  }
+
+  private String getUserName() {
+    return getConf().get(HiveAuthzConf.HIVE_SENTRY_SUBJECT_NAME);
+  }
+
+  /**
+   * load Hive auth provider
+   *
+   * @return
+   * @throws MetaException
+   */
+  private HiveAuthzBinding getHiveAuthzBinding() throws MetaException {
+    if (hiveAuthzBinding == null) {
+      String hiveAuthzConf = getConf().get(HiveAuthzConf.HIVE_SENTRY_CONF_URL);
+      if (hiveAuthzConf == null
+          || (hiveAuthzConf = hiveAuthzConf.trim()).isEmpty()) {
+        throw new MetaException("Configuration key "
+            + HiveAuthzConf.HIVE_SENTRY_CONF_URL + " value '" + hiveAuthzConf
+            + "' is invalid.");
+      }
+      try {
+        authzConf = new HiveAuthzConf(new URL(hiveAuthzConf));
+      } catch (MalformedURLException e) {
+        throw new MetaException("Configuration key "
+            + HiveAuthzConf.HIVE_SENTRY_CONF_URL
+            + " specifies a malformed URL '" + hiveAuthzConf + "' "
+            + e.getMessage());
+      }
+      try {
+        hiveAuthzBinding = new HiveAuthzBinding(
+            HiveAuthzBinding.HiveHook.HiveMetaStore, getConf(), authzConf);
+      } catch (Exception e) {
+        throw new MetaException("Failed to load Hive binding " + e.getMessage());
+      }
+    }
+    return hiveAuthzBinding;
+  }
+
+  private HiveConf getConf() {
+    return SessionState.get().getConf();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sentry/blob/7a30c819/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/SentryMetaStoreFilterHook.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/SentryMetaStoreFilterHook.java b/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/SentryMetaStoreFilterHook.java
new file mode 100644
index 0000000..2a0a5b8
--- /dev/null
+++ b/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/SentryMetaStoreFilterHook.java
@@ -0,0 +1,201 @@
+/**
+ * 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.binding.metastore;
+
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.MetaStoreFilterHook;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.Index;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.PartitionSpec;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.ql.plan.HiveOperation;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.sentry.binding.hive.HiveAuthzBindingHookBase;
+import org.apache.sentry.binding.hive.authz.HiveAuthzBinding;
+import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
+
+public class SentryMetaStoreFilterHook implements MetaStoreFilterHook {
+
+  static final protected Log LOG = LogFactory.getLog(SentryMetaStoreFilterHook.class);
+
+  private HiveAuthzBinding hiveAuthzBinding;
+  private HiveAuthzConf authzConf;
+
+  public SentryMetaStoreFilterHook(HiveConf hiveConf) { //NOPMD
+  }
+
+  @Override
+  public List<String> filterDatabases(List<String> dbList) {
+    return filterDb(dbList);
+  }
+
+  @Override
+  public Database filterDatabase(Database dataBase)
+      throws NoSuchObjectException {
+    return dataBase;
+  }
+
+  @Override
+  public List<String> filterTableNames(String dbName, List<String> tableList) {
+    return filterTab(dbName, tableList);
+  }
+
+  @Override
+  public Table filterTable(Table table) throws NoSuchObjectException {
+    return table;
+  }
+
+  @Override
+  public List<Table> filterTables(List<Table> tableList) {
+    return tableList;
+  }
+
+  @Override
+  public List<Partition> filterPartitions(List<Partition> partitionList) {
+    return partitionList;
+  }
+
+  @Override
+  public List<PartitionSpec> filterPartitionSpecs(
+      List<PartitionSpec> partitionSpecList) {
+    return partitionSpecList;
+  }
+
+  @Override
+  public Partition filterPartition(Partition partition)
+      throws NoSuchObjectException {
+    return partition;
+  }
+
+  @Override
+  public List<String> filterPartitionNames(String dbName, String tblName,
+      List<String> partitionNames) {
+    return partitionNames;
+  }
+
+  @Override
+  public Index filterIndex(Index index) throws NoSuchObjectException {
+    return index;
+  }
+
+  @Override
+  public List<String> filterIndexNames(String dbName, String tblName,
+      List<String> indexList) {
+    return indexList;
+  }
+
+  @Override
+  public List<Index> filterIndexes(List<Index> indexeList) {
+    return indexeList;
+  }
+
+  /**
+   * Invoke Hive database filtering that removes the entries which use has no
+   * privileges to access
+   * @param dbList
+   * @return
+   * @throws MetaException
+   */
+  private List<String> filterDb(List<String> dbList) {
+    try {
+      return HiveAuthzBindingHookBase.filterShowDatabases(getHiveAuthzBinding(),
+          dbList, HiveOperation.SHOWDATABASES, getUserName());
+    } catch (Exception e) {
+      LOG.warn("Error getting DB list ", e);
+      return new ArrayList<String>();
+    } finally {
+      close();
+    }
+  }
+
+  /**
+   * Invoke Hive table filtering that removes the entries which use has no
+   * privileges to access
+   * @param tabList
+   * @return
+   * @throws MetaException
+   */
+  private List<String> filterTab(String dbName, List<String> tabList) {
+    try {
+      return HiveAuthzBindingHookBase.filterShowTables(getHiveAuthzBinding(),
+          tabList, HiveOperation.SHOWTABLES, getUserName(), dbName);
+    } catch (Exception e) {
+      LOG.warn("Error getting Table list ", e);
+      return new ArrayList<String>();
+    } finally {
+      close();
+    }
+  }
+
+  private String getUserName() {
+    return getConf().get(HiveAuthzConf.HIVE_SENTRY_SUBJECT_NAME);
+  }
+
+  /**
+   * load Hive auth provider
+   * @return
+   * @throws MetaException
+   */
+  private HiveAuthzBinding getHiveAuthzBinding() throws MetaException {
+    if (hiveAuthzBinding == null) {
+      String hiveAuthzConf = getConf().get(HiveAuthzConf.HIVE_SENTRY_CONF_URL);
+      if (hiveAuthzConf == null
+          || (hiveAuthzConf = hiveAuthzConf.trim()).isEmpty()) {
+        throw new MetaException("Configuration key "
+            + HiveAuthzConf.HIVE_SENTRY_CONF_URL + " value '" + hiveAuthzConf
+            + "' is invalid.");
+      }
+      try {
+        authzConf = new HiveAuthzConf(new URL(hiveAuthzConf));
+      } catch (MalformedURLException e) {
+        throw new MetaException("Configuration key "
+            + HiveAuthzConf.HIVE_SENTRY_CONF_URL
+            + " specifies a malformed URL '" + hiveAuthzConf + "' "
+            + e.getMessage());
+      }
+      try {
+        hiveAuthzBinding = new HiveAuthzBinding(
+            HiveAuthzBinding.HiveHook.HiveMetaStore, getConf(), authzConf);
+      } catch (Exception e) {
+        throw new MetaException("Failed to load Hive binding " + e.getMessage());
+      }
+    }
+    return hiveAuthzBinding;
+  }
+
+  private HiveConf getConf() {
+    return SessionState.get().getConf();
+  }
+
+  private void close() {
+    if (hiveAuthzBinding != null) {
+      hiveAuthzBinding.close();
+      hiveAuthzBinding = null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/sentry/blob/7a30c819/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/SentryMetastorePostEventListenerBase.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/SentryMetastorePostEventListenerBase.java b/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/SentryMetastorePostEventListenerBase.java
new file mode 100644
index 0000000..d1197a4
--- /dev/null
+++ b/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/SentryMetastorePostEventListenerBase.java
@@ -0,0 +1,404 @@
+/**
+ * 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.binding.metastore;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+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;
+import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
+import org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars;
+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 SentryMetastorePostEventListenerBase extends MetaStoreEventListener {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(SentryMetastoreListenerPlugin.class);
+  private final HiveAuthzConf authzConf;
+  private final Server server;
+
+  protected List<SentryMetastoreListenerPlugin> sentryPlugins = new ArrayList<SentryMetastoreListenerPlugin>();
+
+  public SentryMetastorePostEventListenerBase(Configuration config) {
+    super(config);
+
+    if (!(config instanceof HiveConf)) {
+        String error = "Could not initialize Plugin - Configuration is not an instanceof HiveConf";
+        LOGGER.error(error);
+        throw new RuntimeException(error);
+    }
+
+    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, Configuration.class)
+            .newInstance(config, authzConf);
+        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 {
+
+    // don't sync paths/privileges if the operation has failed
+    if (!tableEvent.getStatus()) {
+      LOGGER.debug("Skip sync paths/privileges with Sentry server for onCreateTable event," +
+        " since the operation failed. \n");
+      return;
+    }
+
+    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)) {
+      return;
+    }
+
+    dropSentryTablePrivilege(tableEvent.getTable().getDbName(),
+        tableEvent.getTable().getTableName());
+  }
+
+  @Override
+  public void onDropTable(DropTableEvent tableEvent) throws MetaException {
+
+    // don't sync paths/privileges if the operation has failed
+    if (!tableEvent.getStatus()) {
+      LOGGER.debug("Skip syncing paths/privileges with Sentry server for onDropTable event," +
+        " since the operation failed. \n");
+      return;
+    }
+
+    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;
+    }
+
+    if (!tableEvent.getStatus()) {
+      return;
+    }
+
+    dropSentryTablePrivilege(tableEvent.getTable().getDbName(),
+        tableEvent.getTable().getTableName());
+  }
+
+  @Override
+  public void onCreateDatabase(CreateDatabaseEvent dbEvent)
+      throws MetaException {
+
+    // don't sync paths/privileges if the operation has failed
+    if (!dbEvent.getStatus()) {
+      LOGGER.debug("Skip syncing paths/privileges with Sentry server for onCreateDatabase event," +
+        " since the operation failed. \n");
+      return;
+    }
+
+    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, in case anything left behind during
+    // last drop db
+    if (!syncWithPolicyStore(AuthzConfVars.AUTHZ_SYNC_CREATE_WITH_POLICY_STORE)) {
+      return;
+    }
+
+    dropSentryDbPrivileges(dbEvent.getDatabase().getName());
+  }
+
+  /**
+   * Drop the privileges on the database. Note that child tables will be
+   * dropped individually by client, so we just need to handle the removing
+   * the db privileges. The table drop should cleanup the table privileges.
+   */
+  @Override
+  public void onDropDatabase(DropDatabaseEvent dbEvent) throws MetaException {
+
+    // don't sync paths/privileges if the operation has failed
+    if (!dbEvent.getStatus()) {
+      LOGGER.debug("Skip syncing paths/privileges with Sentry server for onDropDatabase event," +
+        " since the operation failed. \n");
+      return;
+    }
+
+    String authzObj = dbEvent.getDatabase().getName();
+    for (SentryMetastoreListenerPlugin plugin : sentryPlugins) {
+      List<String> tNames = dbEvent.getHandler().get_all_tables(authzObj);
+      plugin.removeAllPaths(authzObj, tNames);
+    }
+    if (!syncWithPolicyStore(AuthzConfVars.AUTHZ_SYNC_DROP_WITH_POLICY_STORE)) {
+      return;
+    }
+
+    dropSentryDbPrivileges(dbEvent.getDatabase().getName());
+  }
+
+  /**
+   * Adjust the privileges when table is renamed
+   */
+  @Override
+  public void onAlterTable (AlterTableEvent tableEvent) throws MetaException {
+
+    // don't sync privileges if the operation has failed
+    if (!tableEvent.getStatus()) {
+      LOGGER.debug("Skip syncing privileges with Sentry server for onAlterTable event," +
+        " since the operation failed. \n");
+      return;
+    }
+
+    renameSentryTablePrivilege(tableEvent.getOldTable().getDbName(),
+        tableEvent.getOldTable().getTableName(),
+        tableEvent.getOldTable().getSd().getLocation(),
+        tableEvent.getNewTable().getDbName(),
+        tableEvent.getNewTable().getTableName(),
+        tableEvent.getNewTable().getSd().getLocation());
+  }
+
+  @Override
+  public void onAlterPartition(AlterPartitionEvent partitionEvent)
+      throws MetaException {
+
+    // don't sync privileges if the operation has failed
+    if (!partitionEvent.getStatus()) {
+      LOGGER.debug("Skip syncing privileges with Sentry server for onAlterPartition event," +
+        " since the operation failed. \n");
+      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 {
+
+    // don't sync path if the operation has failed
+    if (!partitionEvent.getStatus()) {
+      LOGGER.debug("Skip syncing path with Sentry server for onAddPartition event," +
+        " since the operation failed. \n");
+      return;
+    }
+
+    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 {
+
+    // don't sync path if the operation has failed
+    if (!partitionEvent.getStatus()) {
+      LOGGER.debug("Skip syncing path with Sentry server for onDropPartition event," +
+        " since the operation failed. \n");
+      return;
+    }
+
+    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()
+      throws MetaException {
+    try {
+      return SentryServiceClientFactory.create(authzConf);
+    } catch (Exception e) {
+      throw new MetaException("Failed to connect to Sentry service "
+          + e.getMessage());
+    }
+  }
+
+  private void dropSentryDbPrivileges(String dbName) throws MetaException {
+    List<Authorizable> authorizableTable = new ArrayList<Authorizable>();
+    authorizableTable.add(server);
+    authorizableTable.add(new Database(dbName));
+    try {
+      dropSentryPrivileges(authorizableTable);
+    } catch (SentryUserException e) {
+      throw new MetaException("Failed to remove Sentry policies for drop DB "
+          + dbName + " Error: " + e.getMessage());
+    } catch (IOException e) {
+      throw new MetaException("Failed to find local user " + e.getMessage());
+    }
+
+  }
+
+  private void dropSentryTablePrivilege(String dbName, String tabName)
+      throws MetaException {
+    List<Authorizable> authorizableTable = new ArrayList<Authorizable>();
+    authorizableTable.add(server);
+    authorizableTable.add(new Database(dbName));
+    authorizableTable.add(new Table(tabName));
+
+    try {
+      dropSentryPrivileges(authorizableTable);
+    } catch (SentryUserException e) {
+      throw new MetaException(
+          "Failed to remove Sentry policies for drop table " + dbName + "."
+              + tabName + " Error: " + e.getMessage());
+    } catch (IOException e) {
+      throw new MetaException("Failed to find local user " + e.getMessage());
+    }
+
+  }
+  private void dropSentryPrivileges(
+      List<? extends Authorizable> authorizableTable)
+      throws SentryUserException, IOException, MetaException {
+    String requestorUserName = UserGroupInformation.getCurrentUser()
+        .getShortUserName();
+    SentryPolicyServiceClient sentryClient = getSentryServiceClient();
+    sentryClient.dropPrivileges(requestorUserName, authorizableTable);
+
+    // Close the connection after dropping privileges is done.
+    sentryClient.close();
+  }
+
+  private void renameSentryTablePrivilege(String oldDbName, String oldTabName,
+      String oldPath, String newDbName, String newTabName, String newPath)
+      throws MetaException {
+    List<Authorizable> oldAuthorizableTable = new ArrayList<Authorizable>();
+    oldAuthorizableTable.add(server);
+    oldAuthorizableTable.add(new Database(oldDbName));
+    oldAuthorizableTable.add(new Table(oldTabName));
+
+    List<Authorizable> newAuthorizableTable = new ArrayList<Authorizable>();
+    newAuthorizableTable.add(server);
+    newAuthorizableTable.add(new Database(newDbName));
+    newAuthorizableTable.add(new Table(newTabName));
+
+    if (!oldTabName.equalsIgnoreCase(newTabName)
+        && syncWithPolicyStore(AuthzConfVars.AUTHZ_SYNC_ALTER_WITH_POLICY_STORE)) {
+
+      SentryPolicyServiceClient sentryClient = getSentryServiceClient();
+
+      try {
+        String requestorUserName = UserGroupInformation.getCurrentUser()
+            .getShortUserName();
+        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());
+      } finally {
+
+        // Close the connection after renaming privileges is done.
+        sentryClient.close();
+      }
+    }
+    // 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);
+    }
+  }
+
+  private boolean syncWithPolicyStore(AuthzConfVars syncConfVar) {
+    return "true"
+        .equalsIgnoreCase(authzConf.get(syncConfVar.getVar(), "true"));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sentry/blob/7a30c819/sentry-binding/sentry-binding-hive-v2/pom.xml
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive-v2/pom.xml b/sentry-binding/sentry-binding-hive-v2/pom.xml
index ef6048c..f33219d 100644
--- a/sentry-binding/sentry-binding-hive-v2/pom.xml
+++ b/sentry-binding/sentry-binding-hive-v2/pom.xml
@@ -31,7 +31,7 @@ limitations under the License.
   <dependencies>
     <dependency>
       <groupId>org.apache.sentry</groupId>
-      <artifactId>sentry-binding-hive</artifactId>
+      <artifactId>sentry-binding-hive-common</artifactId>
       <exclusions>
         <exclusion>
           <groupId>org.apache.httpcomponents</groupId>
@@ -114,21 +114,6 @@ limitations under the License.
       <groupId>org.apache.sentry</groupId>
       <artifactId>sentry-provider-common</artifactId>
     </dependency>
-    <!-- required for SentryGrantRevokeTask -->
-    <dependency>
-      <groupId>org.apache.sentry</groupId>
-      <artifactId>sentry-provider-db</artifactId>
-      <exclusions>
-        <exclusion>
-          <groupId>org.apache.hive</groupId>
-          <artifactId>hive-beeline</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.hive</groupId>
-          <artifactId>hive-metastore</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
     <dependency>
       <groupId>org.apache.sentry</groupId>
       <artifactId>sentry-provider-file</artifactId>

http://git-wip-us.apache.org/repos/asf/sentry/blob/7a30c819/sentry-binding/sentry-binding-hive/pom.xml
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/pom.xml b/sentry-binding/sentry-binding-hive/pom.xml
index fb5f214..ff5882e 100644
--- a/sentry-binding/sentry-binding-hive/pom.xml
+++ b/sentry-binding/sentry-binding-hive/pom.xml
@@ -30,14 +30,6 @@ limitations under the License.
 
   <dependencies>
     <dependency>
-      <groupId>org.apache.thrift</groupId>
-      <artifactId>libthrift</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.derby</groupId>
-      <artifactId>derby</artifactId>
-    </dependency>
-    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>
@@ -54,32 +46,7 @@ limitations under the License.
     </dependency>
     <dependency>
       <groupId>org.apache.sentry</groupId>
-      <artifactId>sentry-core-common</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.sentry</groupId>
-      <artifactId>sentry-core-model-db</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.sentry</groupId>
-      <artifactId>sentry-provider-common</artifactId>
-    </dependency>
-    <!-- required for SentryGrantRevokeTask -->
-    <dependency>
-      <groupId>org.apache.sentry</groupId>
-      <artifactId>sentry-provider-db</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.sentry</groupId>
-      <artifactId>sentry-provider-file</artifactId>
-    </dependency>
-      <dependency>
-        <groupId>org.apache.sentry</groupId>
-        <artifactId>sentry-provider-cache</artifactId>
-      </dependency>
-    <dependency>
-      <groupId>org.apache.sentry</groupId>
-      <artifactId>sentry-policy-db</artifactId>
+      <artifactId>sentry-binding-hive-common</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
@@ -97,6 +64,11 @@ limitations under the License.
       <artifactId>mockito-all</artifactId>
       <scope>test</scope>
     </dependency>
+    <!-- required for SentryGrantRevokeTask -->
+    <dependency>
+      <groupId>org.apache.sentry</groupId>
+      <artifactId>sentry-provider-db</artifactId>
+    </dependency>
   </dependencies>
 
 </project>

http://git-wip-us.apache.org/repos/asf/sentry/blob/7a30c819/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/SentryHiveConstants.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/SentryHiveConstants.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/SentryHiveConstants.java
deleted file mode 100644
index 5238414..0000000
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/SentryHiveConstants.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive;
-
-import java.util.EnumSet;
-
-import org.apache.hadoop.hive.ql.security.authorization.PrivilegeType;
-
-public class SentryHiveConstants {
-  public static final EnumSet<PrivilegeType> ALLOWED_PRIVS = EnumSet.allOf(PrivilegeType.class);
-
-  public static final String PRIVILEGE_NOT_SUPPORTED = "Sentry does not support privilege: ";
-  public static final String PARTITION_PRIVS_NOT_SUPPORTED = "Sentry does not support partition level authorization";
-  public static final String GRANT_REVOKE_NOT_SUPPORTED_ON_OBJECT = "Sentry does not allow grant/revoke on: ";
-  public static final String GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL = "Sentry does not allow privileges to be granted/revoked to/from: ";
-}

http://git-wip-us.apache.org/repos/asf/sentry/blob/7a30c819/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryFilterDDLTask.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryFilterDDLTask.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryFilterDDLTask.java
deleted file mode 100644
index 8838368..0000000
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryFilterDDLTask.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive.ql.exec;
-
-import static org.apache.hadoop.util.StringUtils.stringifyException;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.ql.DriverContext;
-import org.apache.hadoop.hive.ql.ErrorMsg;
-import org.apache.hadoop.hive.ql.metadata.Hive;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.metadata.Table;
-import org.apache.hadoop.hive.ql.metadata.formatting.MetaDataFormatUtils;
-import org.apache.hadoop.hive.ql.plan.HiveOperation;
-import org.apache.hadoop.hive.ql.plan.ShowColumnsDesc;
-import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.sentry.binding.hive.HiveAuthzBindingHook;
-import org.apache.sentry.binding.hive.authz.HiveAuthzBinding;
-import org.apache.sentry.core.common.Subject;
-
-import com.google.common.base.Preconditions;
-
-public class SentryFilterDDLTask extends DDLTask {
-  private static final long serialVersionUID = 1L;
-  private static final Log LOG = LogFactory.getLog(SentryFilterDDLTask.class);
-
-  private HiveAuthzBinding hiveAuthzBinding;
-  private Subject subject;
-  private HiveOperation stmtOperation;
-
-  public SentryFilterDDLTask(HiveAuthzBinding hiveAuthzBinding, Subject subject,
-      HiveOperation stmtOperation) {
-    Preconditions.checkNotNull(hiveAuthzBinding);
-    Preconditions.checkNotNull(subject);
-    Preconditions.checkNotNull(stmtOperation);
-
-    this.hiveAuthzBinding = hiveAuthzBinding;
-    this.subject = subject;
-    this.stmtOperation = stmtOperation;
-  }
-
-  public HiveAuthzBinding getHiveAuthzBinding() {
-    return hiveAuthzBinding;
-  }
-
-  public Subject getSubject() {
-    return subject;
-  }
-
-  public HiveOperation getStmtOperation() {
-    return stmtOperation;
-  }
-
-  @Override
-  public int execute(DriverContext driverContext) {
-    // Currently the SentryFilterDDLTask only supports filter the "show columns in table " command.
-    ShowColumnsDesc showCols = work.getShowColumnsDesc();
-    try {
-      if (showCols != null) {
-        return showFilterColumns(showCols);
-      }
-    } catch (Throwable e) {
-      failed(e);
-      return 1;
-    }
-
-    return super.execute(driverContext);
-  }
-
-  private void failed(Throwable e) {
-    while (e.getCause() != null && e.getClass() == RuntimeException.class) {
-      e = e.getCause();
-    }
-    setException(e);
-    LOG.error(stringifyException(e));
-  }
-
-  /**
-   * Filter the command "show columns in table"
-   *
-   */
-  private int showFilterColumns(ShowColumnsDesc showCols) throws HiveException {
-    Table table = Hive.get(conf).getTable(showCols.getTableName());
-
-    // write the results in the file
-    DataOutputStream outStream = null;
-    try {
-      Path resFile = new Path(showCols.getResFile());
-      FileSystem fs = resFile.getFileSystem(conf);
-      outStream = fs.create(resFile);
-
-      List<FieldSchema> cols = table.getCols();
-      cols.addAll(table.getPartCols());
-      // In case the query is served by HiveServer2, don't pad it with spaces,
-      // as HiveServer2 output is consumed by JDBC/ODBC clients.
-      boolean isOutputPadded = !SessionState.get().isHiveServerQuery();
-      outStream.writeBytes(MetaDataFormatUtils.getAllColumnsInformation(
-          fiterColumns(cols, table), false, isOutputPadded, null));
-      outStream.close();
-      outStream = null;
-    } catch (IOException e) {
-      throw new HiveException(e, ErrorMsg.GENERIC_ERROR);
-    } finally {
-      IOUtils.closeStream(outStream);
-    }
-    return 0;
-  }
-
-  private List<FieldSchema> fiterColumns(List<FieldSchema> cols, Table table) throws HiveException {
-    // filter some columns that the subject has privilege on
-    return HiveAuthzBindingHook.filterShowColumns(getHiveAuthzBinding(),
-        cols, getStmtOperation(), getSubject().getName(), table.getTableName(), table.getDbName());
-  }
-}

http://git-wip-us.apache.org/repos/asf/sentry/blob/7a30c819/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
index 31eb5e8..cd8352a 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
@@ -55,7 +55,7 @@ import org.apache.hadoop.hive.ql.security.authorization.PrivilegeType;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.session.SessionState.LogHelper;
 import org.apache.sentry.SentryUserException;
-import org.apache.sentry.binding.hive.HiveAuthzBindingHook;
+import org.apache.sentry.binding.hive.HiveAuthzBindingHookBase;
 import org.apache.sentry.binding.hive.SentryOnFailureHookContext;
 import org.apache.sentry.binding.hive.SentryOnFailureHookContextImpl;
 import org.apache.sentry.binding.hive.authz.HiveAuthzBinding;
@@ -160,7 +160,7 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
             new HashSet<WriteEntity>(), stmtOperation,
             null, null, null, null, subject.getName(), ipAddress,
             new AuthorizationException(e), conf);
-        HiveAuthzBindingHook.runFailureHook(hookContext, csHooks);
+        HiveAuthzBindingHookBase.runFailureHook(hookContext, csHooks);
         throw e; // rethrow the exception for logging
       }
     } catch(SentryUserException e) {

http://git-wip-us.apache.org/repos/asf/sentry/blob/7a30c819/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryHivePrivilegeObjectDesc.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryHivePrivilegeObjectDesc.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryHivePrivilegeObjectDesc.java
deleted file mode 100644
index 4fa4221..0000000
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryHivePrivilegeObjectDesc.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.ql.exec;
-
-import org.apache.hadoop.hive.ql.plan.PrivilegeObjectDesc;
-
-public class SentryHivePrivilegeObjectDesc extends PrivilegeObjectDesc {
-  private boolean isUri;
-  private boolean isServer;
-
-  public SentryHivePrivilegeObjectDesc() {
-    // reset table type which is on by default
-    super.setTable(false);
-  }
-
-  public boolean getUri() {
-    return isUri;
-  }
-
-  public void setUri(boolean isUri) {
-    this.isUri = isUri;
-  }
-
-  public boolean getServer() {
-    return isServer;
-  }
-
-  public void setServer(boolean isServer) {
-    this.isServer = isServer;
-  }
-
-  public boolean isSentryPrivObjectDesc() {
-    return isServer || isUri;
-  }
-
-}