http://git-wip-us.apache.org/repos/asf/sentry/blob/b19cb01b/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
deleted file mode 100644
index 3e2a9ea..0000000
--- 
a/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/MetastoreAuthzBindingBase.java
+++ /dev/null
@@ -1,453 +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.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 = null;
-    String newLocationUri = null;
-    try {
-      if (!StringUtils.isEmpty(context.getOldTable().getSd().getLocation())) {
-        oldLocationUri = PathUtils.parseDFSURI(warehouseDir,
-            getSdLocation(context.getOldTable().getSd()));
-      }
-      if (!StringUtils.isEmpty(context.getNewTable().getSd().getLocation())) {
-        newLocationUri = PathUtils.parseDFSURI(warehouseDir,
-            getSdLocation(context.getNewTable().getSd()));
-      }
-    } catch (URISyntaxException e) {
-      throw new MetaException(e.getMessage());
-    }
-    if (!StringUtils.equals(oldLocationUri, newLocationUri)) {
-      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/b19cb01b/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
deleted file mode 100644
index b5df287..0000000
--- 
a/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/SentryHiveMetaStoreClient.java
+++ /dev/null
@@ -1,161 +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.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/b19cb01b/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
deleted file mode 100644
index 2a0a5b8..0000000
--- 
a/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/SentryMetaStoreFilterHook.java
+++ /dev/null
@@ -1,201 +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.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/b19cb01b/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
deleted file mode 100644
index 2abdd53..0000000
--- 
a/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/SentryMetastorePostEventListenerBase.java
+++ /dev/null
@@ -1,409 +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.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.core.common.exception.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;
-    }
-    String oldLoc = null, newLoc = null;
-    org.apache.hadoop.hive.metastore.api.Table oldTal = 
tableEvent.getOldTable();
-    org.apache.hadoop.hive.metastore.api.Table newTal = 
tableEvent.getNewTable();
-    if (oldTal != null && oldTal.getSd() != null) {
-      oldLoc = oldTal.getSd().getLocation();
-    }
-    if (newTal != null && newTal.getSd() != null) {
-      newLoc = newTal.getSd().getLocation();
-    }
-    if (oldLoc != null && newLoc != null && !oldLoc.equals(newLoc)) {
-      String oldDbName = tableEvent.getOldTable().getDbName();
-      String oldTbName = tableEvent.getOldTable().getTableName();
-      String newTbName = tableEvent.getNewTable().getTableName();
-      String newDbName = tableEvent.getNewTable().getDbName();
-      renameSentryTablePrivilege(oldDbName, oldTbName, oldLoc, newDbName, 
newTbName, newLoc);
-    }
-  }
-
-  @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();
-    try(SentryPolicyServiceClient sentryClient = getSentryServiceClient()) {
-      sentryClient.dropPrivileges(requestorUserName, authorizableTable);
-    } catch (Exception e) {
-      e.printStackTrace();
-    }
-  }
-
-  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)) {
-
-      try (SentryPolicyServiceClient sentryClient = getSentryServiceClient()){
-        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());
-      } catch (Exception e) {
-        e.printStackTrace();
-      }
-    }
-    // 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/b19cb01b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/hadoop/hive/ql/exec/SentryFilterDDLTask.java
----------------------------------------------------------------------
diff --git 
a/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/hadoop/hive/ql/exec/SentryFilterDDLTask.java
 
b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/hadoop/hive/ql/exec/SentryFilterDDLTask.java
new file mode 100644
index 0000000..0b1acf1
--- /dev/null
+++ 
b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/hadoop/hive/ql/exec/SentryFilterDDLTask.java
@@ -0,0 +1,162 @@
+/*
+ * 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.HiveAuthzBindingHookBaseV2;
+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) {
+    // Get the cause of the exception if available
+    Throwable error = e;
+    while (error.getCause() != null && error.getClass() == 
RuntimeException.class) {
+      error = error.getCause();
+    }
+    setException(error);
+    LOG.error(stringifyException(error));
+  }
+
+  /**
+   * 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 HiveAuthzBindingHookBaseV2.filterShowColumns(getHiveAuthzBinding(),
+        cols, getStmtOperation(), getSubject().getName(), 
table.getTableName(), table.getDbName());
+  }
+
+  public void copyDDLTask(DDLTask ddlTask) {
+    work = ddlTask.getWork();
+    rootTask = ddlTask.isRootTask();
+    childTasks = ddlTask.getChildTasks();
+    parentTasks = ddlTask.getParentTasks();
+    backupTask = ddlTask.getBackupTask();
+    backupChildrenTasks = ddlTask.getBackupChildrenTasks();
+    started = ddlTask.started();
+    isdone = ddlTask.done();
+    queued = ddlTask.getQueued();
+    id = ddlTask.getId();
+    taskCounters = ddlTask.getCounters();
+    feedSubscribers = ddlTask.getFeedSubscribers();
+    taskTag = ddlTask.getTaskTag();
+    setLocalMode(ddlTask.isLocalMode());
+    setRetryCmdWhenFail(ddlTask.ifRetryCmdWhenFail());
+    queryPlan = ddlTask.getQueryPlan();
+    jobID = ddlTask.getJobID();
+    setException(ddlTask.getException());
+    console = ddlTask.console;
+    setFetchSource(ddlTask.isFetchSource());
+  }
+}

http://git-wip-us.apache.org/repos/asf/sentry/blob/b19cb01b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/authz/SentryConfigTool.java
----------------------------------------------------------------------
diff --git 
a/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/authz/SentryConfigTool.java
 
b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/authz/SentryConfigTool.java
new file mode 100644
index 0000000..1dc8f01
--- /dev/null
+++ 
b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/authz/SentryConfigTool.java
@@ -0,0 +1,643 @@
+/*
+ * 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.hive.authz;
+
+import java.security.CodeSource;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionGroup;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.Parser;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.log4j.Level;
+import org.apache.log4j.LogManager;
+import org.apache.sentry.Command;
+import org.apache.sentry.binding.hive.HiveAuthzBindingHookBaseV2;
+import org.apache.sentry.binding.hive.SentryPolicyFileFormatFactory;
+import org.apache.sentry.binding.hive.SentryPolicyFileFormatter;
+import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
+import org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars;
+import org.apache.sentry.core.common.exception.SentryConfigurationException;
+import org.apache.sentry.core.common.Subject;
+import org.apache.sentry.core.model.db.Server;
+import org.apache.sentry.provider.common.AuthorizationProvider;
+import org.apache.sentry.provider.db.service.thrift.SentryPolicyServiceClient;
+import org.apache.sentry.service.thrift.SentryServiceClientFactory;
+
+/**
+ * set the required system property to be read by HiveConf and AuthzConf
+ *
+ * @throws Exception
+ */
+// Hack, hiveConf doesn't provide a reliable way check if it found a valid
+// hive-site
+// load auth provider
+// get the configured sentry provider
+// validate policy files
+// import policy files
+public class SentryConfigTool {
+  private String sentrySiteFile = null;
+  private String policyFile = null;
+  private String query = null;
+  private String jdbcURL = null;
+  private String user = null;
+  private String passWord = null;
+  private String importPolicyFilePath = null;
+  private String exportPolicyFilePath = null;
+  private String objectPath = null;
+  private boolean listPrivs = false;
+  private boolean validate = false;
+  private boolean importOverwriteRole = false;
+  private HiveConf hiveConf = null;
+  private HiveAuthzConf authzConf = null;
+  private AuthorizationProvider sentryProvider = null;
+
+  public SentryConfigTool() {
+
+  }
+
+  public AuthorizationProvider getSentryProvider() {
+    return sentryProvider;
+  }
+
+  public void setSentryProvider(AuthorizationProvider sentryProvider) {
+    this.sentryProvider = sentryProvider;
+  }
+
+  public HiveConf getHiveConf() {
+    return hiveConf;
+  }
+
+  public void setHiveConf(HiveConf hiveConf) {
+    this.hiveConf = hiveConf;
+  }
+
+  public HiveAuthzConf getAuthzConf() {
+    return authzConf;
+  }
+
+  public void setAuthzConf(HiveAuthzConf authzConf) {
+    this.authzConf = authzConf;
+  }
+
+  public boolean isValidate() {
+    return validate;
+  }
+
+  public void setValidate(boolean validate) {
+    this.validate = validate;
+  }
+
+  public String getImportPolicyFilePath() {
+    return importPolicyFilePath;
+  }
+
+  public void setImportPolicyFilePath(String importPolicyFilePath) {
+    this.importPolicyFilePath = importPolicyFilePath;
+  }
+
+  public String getObjectPath() {
+    return objectPath;
+  }
+
+  public void setObjectPath(String objectPath) {
+    this.objectPath = objectPath;
+  }
+
+  public String getExportPolicyFilePath() {
+    return exportPolicyFilePath;
+  }
+
+  public void setExportPolicyFilePath(String exportPolicyFilePath) {
+    this.exportPolicyFilePath = exportPolicyFilePath;
+  }
+
+  public String getSentrySiteFile() {
+    return sentrySiteFile;
+  }
+
+  public void setSentrySiteFile(String sentrySiteFile) {
+    this.sentrySiteFile = sentrySiteFile;
+  }
+
+  public String getPolicyFile() {
+    return policyFile;
+  }
+
+  public void setPolicyFile(String policyFile) {
+    this.policyFile = policyFile;
+  }
+
+  public String getQuery() {
+    return query;
+  }
+
+  public void setQuery(String query) {
+    this.query = query;
+  }
+
+  public String getJdbcURL() {
+    return jdbcURL;
+  }
+
+  public void setJdbcURL(String jdbcURL) {
+    this.jdbcURL = jdbcURL;
+  }
+
+  public String getUser() {
+    return user;
+  }
+
+  public void setUser(String user) {
+    this.user = user;
+  }
+
+  public String getPassWord() {
+    return passWord;
+  }
+
+  public void setPassWord(String passWord) {
+    this.passWord = passWord;
+  }
+
+  public boolean isListPrivs() {
+    return listPrivs;
+  }
+
+  public void setListPrivs(boolean listPrivs) {
+    this.listPrivs = listPrivs;
+  }
+
+  public boolean isImportOverwriteRole() {
+    return importOverwriteRole;
+  }
+
+  public void setImportOverwriteRole(boolean importOverwriteRole) {
+    this.importOverwriteRole = importOverwriteRole;
+  }
+
+  /**
+   * set the required system property to be read by HiveConf and AuthzConf
+   * @throws Exception
+   */
+  public void setupConfig() throws Exception {
+    System.out.println("Configuration: ");
+    CodeSource src = SentryConfigTool.class.getProtectionDomain()
+        .getCodeSource();
+    if (src != null) {
+      System.out.println("Sentry package jar: " + src.getLocation());
+    }
+
+    if (getPolicyFile() != null) {
+      System.setProperty(AuthzConfVars.AUTHZ_PROVIDER_RESOURCE.getVar(),
+          getPolicyFile());
+    }
+    System.setProperty(AuthzConfVars.SENTRY_TESTING_MODE.getVar(), "true");
+    setHiveConf(new HiveConf(SessionState.class));
+    getHiveConf().setVar(ConfVars.SEMANTIC_ANALYZER_HOOK,
+        HiveAuthzBindingHookBaseV2.class.getName());
+    try {
+      System.out.println("Hive config: " + HiveConf.getHiveSiteLocation());
+    } catch (NullPointerException e) {
+      // Hack, hiveConf doesn't provide a reliable way check if it found a 
valid
+      // hive-site
+      throw new SentryConfigurationException("Didn't find a hive-site.xml");
+
+    }
+
+    if (getSentrySiteFile() != null) {
+      getHiveConf()
+          .set(HiveAuthzConf.HIVE_SENTRY_CONF_URL, getSentrySiteFile());
+    }
+
+    setAuthzConf(HiveAuthzConf.getAuthzConf(getHiveConf()));
+    System.out.println("Sentry config: "
+        + getAuthzConf().getHiveAuthzSiteFile());
+    System.out.println("Sentry Policy: "
+        + getAuthzConf().get(AuthzConfVars.AUTHZ_PROVIDER_RESOURCE.getVar()));
+    System.out.println("Sentry server: "
+        + getAuthzConf().get(AuthzConfVars.AUTHZ_SERVER_NAME.getVar()));
+
+    setSentryProvider(getAuthorizationProvider());
+  }
+
+  // load auth provider
+  private AuthorizationProvider getAuthorizationProvider()
+      throws IllegalStateException, SentryConfigurationException {
+    String serverName = new Server(getAuthzConf().get(
+        AuthzConfVars.AUTHZ_SERVER_NAME.getVar())).getName();
+    // get the configured sentry provider
+    try {
+      return HiveAuthzBinding.getAuthProvider(getHiveConf(),
+          authzConf, serverName);
+    } catch (SentryConfigurationException eC) {
+      printConfigErrors(eC);
+      throw eC;
+    } catch (Exception e) {
+      throw new IllegalStateException("Couldn't load sentry provider ", e);
+    }
+  }
+
+  // validate policy files
+  public void validatePolicy() throws Exception {
+    try {
+      getSentryProvider().validateResource(true);
+    } catch (SentryConfigurationException e) {
+      printConfigErrors(e);
+      throw e;
+    }
+    System.out.println("No errors found in the policy file");
+  }
+
+  // import the sentry mapping data to database
+  public void importPolicy() throws Exception {
+    String requestorUserName = System.getProperty("user.name", "");
+    // get the FileFormatter according to the configuration
+    SentryPolicyFileFormatter sentryPolicyFileFormatter = 
SentryPolicyFileFormatFactory
+        .createFileFormatter(authzConf);
+    // parse the input file, get the mapping data in map structure
+    Map<String, Map<String, Set<String>>> policyFileMappingData = 
sentryPolicyFileFormatter.parse(
+        importPolicyFilePath, authzConf);
+    // todo: here should be an validator to check the data's value, format, 
hierarchy
+    try(SentryPolicyServiceClient client =
+                SentryServiceClientFactory.create(getAuthzConf())) {
+      // import the mapping data to database
+      client.importPolicy(policyFileMappingData, requestorUserName, 
importOverwriteRole);
+    }
+  }
+
+  // export the sentry mapping data to file
+  public void exportPolicy() throws Exception {
+    String requestorUserName = System.getProperty("user.name", "");
+    try (SentryPolicyServiceClient client =
+                SentryServiceClientFactory.create(getAuthzConf())) {
+      // export the sentry mapping data from database to map structure
+      Map<String, Map<String, Set<String>>> policyFileMappingData = client
+              .exportPolicy(requestorUserName, objectPath);
+      // get the FileFormatter according to the configuration
+      SentryPolicyFileFormatter sentryPolicyFileFormatter = 
SentryPolicyFileFormatFactory
+              .createFileFormatter(authzConf);
+      // write the sentry mapping data to exportPolicyFilePath with the data 
in map structure
+      sentryPolicyFileFormatter.write(exportPolicyFilePath, 
policyFileMappingData);
+    }
+  }
+
+  // list permissions for given user
+  public void listPrivs() throws Exception {
+    getSentryProvider().validateResource(true);
+    System.out.println("Available privileges for user " + getUser() + ":");
+    Set<String> permList = getSentryProvider().listPrivilegesForSubject(
+        new Subject(getUser()));
+    for (String perms : permList) {
+      System.out.println("\t" + perms);
+    }
+    if (permList.isEmpty()) {
+      System.out.println("\t*** No permissions available ***");
+    }
+  }
+
+  // Verify the given query
+  public void verifyLocalQuery(String queryStr) throws Exception {
+    // setup Hive driver
+    SessionState session = new SessionState(getHiveConf());
+    SessionState.start(session);
+    Driver driver = new Driver(session.getConf(), getUser());
+
+    // compile the query
+    CommandProcessorResponse compilerStatus = driver
+        .compileAndRespond(queryStr);
+    if (compilerStatus.getResponseCode() != 0) {
+      String errMsg = compilerStatus.getErrorMessage();
+      if (errMsg.contains(HiveAuthzConf.HIVE_SENTRY_PRIVILEGE_ERROR_MESSAGE)) {
+        printMissingPerms(getHiveConf().get(
+            HiveAuthzConf.HIVE_SENTRY_AUTH_ERRORS));
+      }
+      throw new SemanticException("Compilation error: "
+          + compilerStatus.getErrorMessage());
+    }
+    driver.close();
+    System.out
+        .println("User " + getUser() + " has privileges to run the query");
+  }
+
+  // connect to remote HS2 and run mock query
+  public void verifyRemoteQuery(String queryStr) throws Exception {
+    Class.forName("org.apache.hive.jdbc.HiveDriver");
+    Connection conn = DriverManager.getConnection(getJdbcURL(), getUser(),
+        getPassWord());
+    Statement stmt = conn.createStatement();
+    if (!isSentryEnabledOnHiveServer(stmt)) {
+      throw new IllegalStateException("Sentry is not enabled on HiveServer2");
+    }
+    stmt.execute("set " + HiveAuthzConf.HIVE_SENTRY_MOCK_COMPILATION + 
"=true");
+    try {
+      stmt.execute(queryStr);
+    } catch (SQLException e) {
+      String errMsg = e.getMessage();
+      if (errMsg.contains(HiveAuthzConf.HIVE_SENTRY_MOCK_ERROR)) {
+        System.out.println("User "
+            + readConfig(stmt, HiveAuthzConf.HIVE_SENTRY_SUBJECT_NAME)
+            + " has privileges to run the query");
+        return;
+      } else if (errMsg
+          .contains(HiveAuthzConf.HIVE_SENTRY_PRIVILEGE_ERROR_MESSAGE)) {
+        printMissingPerms(readConfig(stmt,
+            HiveAuthzConf.HIVE_SENTRY_AUTH_ERRORS));
+        throw e;
+      } else {
+        throw e;
+      }
+    } finally {
+      if (!stmt.isClosed()) {
+        stmt.close();
+      }
+      conn.close();
+    }
+
+  }
+
+  // verify senty session hook is set
+  private boolean isSentryEnabledOnHiveServer(Statement stmt)
+      throws SQLException {
+    String bindingString = readConfig(stmt, 
HiveConf.ConfVars.HIVE_SERVER2_SESSION_HOOK.varname).toUpperCase();
+    return 
bindingString.contains("org.apache.sentry.binding.hive".toUpperCase())
+        && bindingString.contains("HiveAuthzBindingSessionHook".toUpperCase());
+  }
+
+  // read a config value using 'set' statement
+  private String readConfig(Statement stmt, String configKey)
+      throws SQLException {
+    try (ResultSet res = stmt.executeQuery("set " + configKey)) {
+      if (!res.next()) {
+        return null;
+      }
+      // parse key=value result format
+      String result = res.getString(1);
+      res.close();
+      return result.substring(result.indexOf("=") + 1);
+    }
+  }
+
+  // print configuration/policy file errors and warnings
+  private void printConfigErrors(SentryConfigurationException configException)
+      throws SentryConfigurationException {
+    System.out.println(" *** Found configuration problems *** ");
+    for (String errMsg : configException.getConfigErrors()) {
+      System.out.println("ERROR: " + errMsg);
+    }
+    for (String warnMsg : configException.getConfigWarnings()) {
+      System.out.println("Warning: " + warnMsg);
+    }
+  }
+
+  // extract the authorization errors from config property and print
+  private void printMissingPerms(String errMsg) {
+    if (errMsg == null || errMsg.isEmpty()) {
+      return;
+    }
+    System.out.println("*** Query compilation failed ***");
+    String perms[] = errMsg.replaceFirst(
+        ".*" + HiveAuthzConf.HIVE_SENTRY_PRIVILEGE_ERROR_MESSAGE, "")
+        .split(";");
+    System.out.println("Required privileges for given query:");
+    for (int count = 0; count < perms.length; count++) {
+      System.out.println(" \t " + perms[count]);
+    }
+  }
+
+  // print usage
+  private void usage(Options sentryOptions) {
+    HelpFormatter formatter = new HelpFormatter();
+    formatter.printHelp("sentry --command config-tool", sentryOptions);
+    System.exit(-1);
+  }
+
+  /**
+   * parse arguments
+   *
+   * <pre>
+   *   -d,--debug                  Enable debug output
+   *   -e,--query <arg>            Query privilege verification, requires -u
+   *   -h,--help                   Print usage
+   *   -i,--policyIni <arg>        Policy file path
+   *   -j,--jdbcURL <arg>          JDBC URL
+   *   -l,--listPrivs,--listPerms  List privilges for given user, requires -u
+   *   -p,--password <arg>         Password
+   *   -s,--sentry-site <arg>      sentry-site file path
+   *   -u,--user <arg>             user name
+   *   -v,--validate               Validate policy file
+   *   -I,--import                 Import policy file
+   *   -E,--export                 Export policy file
+   *   -o,--overwrite              Overwrite the exist role data when do the 
import
+   *   -b,--objectPath             The path of the object whose privileges 
will be exported
+   * </pre>
+   *
+   * @param args
+   */
+  private void parseArgs(String[] args) {
+    boolean enableDebug = false;
+
+    Options sentryOptions = new Options();
+
+    Option helpOpt = new Option("h", "help", false, "Print usage");
+    helpOpt.setRequired(false);
+
+    Option validateOpt = new Option("v", "validate", false,
+        "Validate policy file");
+    validateOpt.setRequired(false);
+
+    Option queryOpt = new Option("e", "query", true,
+        "Query privilege verification, requires -u");
+    queryOpt.setRequired(false);
+
+    Option listPermsOpt = new Option("l", "listPerms", false,
+        "list permissions for given user, requires -u");
+    listPermsOpt.setRequired(false);
+    Option listPrivsOpt = new Option("listPrivs", false,
+        "list privileges for given user, requires -u");
+    listPrivsOpt.setRequired(false);
+
+    Option importOpt = new Option("I", "import", true,
+        "Import policy file");
+    importOpt.setRequired(false);
+
+    Option exportOpt = new Option("E", "export", true, "Export policy file");
+    exportOpt.setRequired(false);
+    // required args
+    OptionGroup sentryOptGroup = new OptionGroup();
+    sentryOptGroup.addOption(helpOpt);
+    sentryOptGroup.addOption(validateOpt);
+    sentryOptGroup.addOption(queryOpt);
+    sentryOptGroup.addOption(listPermsOpt);
+    sentryOptGroup.addOption(listPrivsOpt);
+    sentryOptGroup.addOption(importOpt);
+    sentryOptGroup.addOption(exportOpt);
+    sentryOptGroup.setRequired(true);
+    sentryOptions.addOptionGroup(sentryOptGroup);
+
+    // optional args
+    Option jdbcArg = new Option("j", "jdbcURL", true, "JDBC URL");
+    jdbcArg.setRequired(false);
+    sentryOptions.addOption(jdbcArg);
+
+    Option sentrySitePath = new Option("s", "sentry-site", true,
+        "sentry-site file path");
+    sentrySitePath.setRequired(false);
+    sentryOptions.addOption(sentrySitePath);
+
+    Option globalPolicyPath = new Option("i", "policyIni", true,
+        "Policy file path");
+    globalPolicyPath.setRequired(false);
+    sentryOptions.addOption(globalPolicyPath);
+
+    Option userOpt = new Option("u", "user", true, "user name");
+    userOpt.setRequired(false);
+    sentryOptions.addOption(userOpt);
+
+    Option passWordOpt = new Option("p", "password", true, "Password");
+    userOpt.setRequired(false);
+    sentryOptions.addOption(passWordOpt);
+
+    Option debugOpt = new Option("d", "debug", false, "enable debug output");
+    debugOpt.setRequired(false);
+    sentryOptions.addOption(debugOpt);
+
+    Option overwriteOpt = new Option("o", "overwrite", false, "enable import 
overwrite");
+    overwriteOpt.setRequired(false);
+    sentryOptions.addOption(overwriteOpt);
+
+    Option objectPathOpt = new Option("b", "objectPath",
+        false, "The path of the object whose privileges will be exported");
+    objectPathOpt.setRequired(false);
+    sentryOptions.addOption(objectPathOpt);
+
+    try {
+      Parser parser = new GnuParser();
+      CommandLine cmd = parser.parse(sentryOptions, args);
+
+      for (Option opt : cmd.getOptions()) {
+        if (opt.getOpt().equals("s")) {
+          setSentrySiteFile(opt.getValue());
+        } else if (opt.getOpt().equals("i")) {
+          setPolicyFile(opt.getValue());
+        } else if (opt.getOpt().equals("e")) {
+          setQuery(opt.getValue());
+        } else if (opt.getOpt().equals("j")) {
+          setJdbcURL(opt.getValue());
+        } else if (opt.getOpt().equals("u")) {
+          setUser(opt.getValue());
+        } else if (opt.getOpt().equals("p")) {
+          setPassWord(opt.getValue());
+        } else if (opt.getOpt().equals("l") || 
opt.getOpt().equals("listPrivs")) {
+          setListPrivs(true);
+        } else if (opt.getOpt().equals("v")) {
+          setValidate(true);
+        } else if (opt.getOpt().equals("I")) {
+          setImportPolicyFilePath(opt.getValue());
+        } else if (opt.getOpt().equals("E")) {
+          setExportPolicyFilePath(opt.getValue());
+        } else if (opt.getOpt().equals("h")) {
+          usage(sentryOptions);
+        } else if (opt.getOpt().equals("d")) {
+          enableDebug = true;
+        } else if (opt.getOpt().equals("o")) {
+          setImportOverwriteRole(true);
+        } else if (opt.getOpt().equals("b")) {
+          setObjectPath(opt.getValue());
+        }
+      }
+
+      if (isListPrivs() && getUser() == null) {
+        throw new ParseException("Can't use -l without -u ");
+      }
+      if (getQuery() != null && getUser() == null) {
+        throw new ParseException("Must use -u with -e ");
+      }
+    } catch (ParseException e1) {
+      usage(sentryOptions);
+    }
+
+    if (!enableDebug) {
+      // turn off log
+      LogManager.getRootLogger().setLevel(Level.OFF);
+    }
+  }
+
+  public static class CommandImpl implements Command {
+    @Override
+    public void run(String[] args) throws Exception {
+      SentryConfigTool sentryTool = new SentryConfigTool();
+
+      try {
+        // parse arguments
+        sentryTool.parseArgs(args);
+
+        // load configuration
+        sentryTool.setupConfig();
+
+        // validate configuration
+        if (sentryTool.isValidate()) {
+          sentryTool.validatePolicy();
+        }
+
+        if (!StringUtils.isEmpty(sentryTool.getImportPolicyFilePath())) {
+          sentryTool.importPolicy();
+        }
+
+        if (!StringUtils.isEmpty(sentryTool.getExportPolicyFilePath())) {
+          sentryTool.exportPolicy();
+        }
+
+        // list permissions for give user
+        if (sentryTool.isListPrivs()) {
+          sentryTool.listPrivs();
+        }
+
+        // verify given query
+        if (sentryTool.getQuery() != null) {
+          if (sentryTool.getJdbcURL() != null) {
+            sentryTool.verifyRemoteQuery(sentryTool.getQuery());
+          } else {
+            sentryTool.verifyLocalQuery(sentryTool.getQuery());
+          }
+        }
+      } catch (Exception e) {
+        System.out.println("Sentry tool reported Errors: " + e.getMessage());
+        e.printStackTrace(System.out);
+        System.exit(1);
+      }
+    }
+  }
+}

Reply via email to