You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by sp...@apache.org on 2018/12/21 14:54:15 UTC

sentry git commit: SENTRY-2481: Filter HMS server-side objects based on HMS user authorization (Sergio Pena, reviewed by Na Li, Arjun Mishra)

Repository: sentry
Updated Branches:
  refs/heads/master 5bfa963b7 -> 3ba5998b8


SENTRY-2481: Filter HMS server-side objects based on HMS user authorization (Sergio Pena, reviewed by Na Li, Arjun Mishra)


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

Branch: refs/heads/master
Commit: 3ba5998b87595ea158d1d3b71b00cbcdb4092a0a
Parents: 5bfa963
Author: Sergio Pena <se...@cloudera.com>
Authored: Fri Dec 21 08:53:39 2018 -0600
Committer: Sergio Pena <se...@cloudera.com>
Committed: Fri Dec 21 08:54:04 2018 -0600

----------------------------------------------------------------------
 .gitignore                                      |   1 +
 pom.xml                                         |   8 +
 .../binding/hive/authz/HiveAuthzBinding.java    |   2 +-
 .../binding/hive/authz/HiveAuthzPrivileges.java |  42 +++
 .../sentry/binding/hive/conf/HiveAuthzConf.java |   7 +
 sentry-binding/sentry-binding-hive/pom.xml      |   5 +
 .../hive/authz/DefaultSentryValidator.java      | 127 ++------
 .../hive/authz/MetastoreAuthzObjectFilter.java  | 189 +++++++++++
 .../metastore/AuthorizingObjectStore.java       |  44 ++-
 .../metastore/AuthorizingObjectStoreBase.java   |  44 ++-
 .../metastore/HiveAuthzBindingFactory.java      |  30 ++
 .../metastore/MetastoreAuthzBindingBase.java    |   2 +-
 .../metastore/SentryHiveMetaStoreClient.java    |  41 ++-
 .../metastore/SentryMetaStoreFilterHook.java    | 227 ++++++++++---
 .../authz/TestMetastoreAuthzObjectFilter.java   | 323 +++++++++++++++++++
 .../TestSentryMetaStoreFilterHook.java          | 219 +++++++++++++
 .../org/apache/sentry/core/common/Subject.java  |  18 ++
 17 files changed, 1144 insertions(+), 185 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sentry/blob/3ba5998b/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 6ce3a6c..419b5b5 100644
--- a/.gitignore
+++ b/.gitignore
@@ -23,3 +23,4 @@ maven-repo/
 **/thirdparty/*
 **/metastore_db/*
 *dependency-reduced-pom.xml
+*.attach*

http://git-wip-us.apache.org/repos/asf/sentry/blob/3ba5998b/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index f28be5a..c0572dc 100644
--- a/pom.xml
+++ b/pom.xml
@@ -101,6 +101,7 @@ limitations under the License.
     <zookeeper.version>3.4.5</zookeeper.version>
     <maven.jar.plugin.version>3.0.2</maven.jar.plugin.version>
     <httpcomponents.version>4.5.3</httpcomponents.version>
+    <assertj.version>3.11.1</assertj.version>
 
     <!-- Package versions for Sentry binding components -->
     <hadoop.version>3.1.1</hadoop.version>
@@ -967,6 +968,12 @@ limitations under the License.
           </exclusion>
         </exclusions>
       </dependency>
+      <dependency>
+        <groupId>org.assertj</groupId>
+        <artifactId>assertj-core</artifactId>
+        <version>${assertj.version}</version>
+        <scope>test</scope>
+      </dependency>
     </dependencies>
   </dependencyManagement>
 
@@ -1197,6 +1204,7 @@ limitations under the License.
                   <exclude>**/*.woff2</exclude>
                   <!-- Lombok Config File -->
                   <exclude>lombok.config</exclude>
+                  <exclude>**/*.attach*</exclude>
                 </excludes>
               </configuration>
             </execution>

http://git-wip-us.apache.org/repos/asf/sentry/blob/3ba5998b/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java b/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java
index 520de52..0397f87 100644
--- a/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java
+++ b/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java
@@ -56,7 +56,7 @@ import com.google.common.base.Splitter;
 import com.google.common.base.Strings;
 import com.google.common.collect.Sets;
 
-public class HiveAuthzBinding {
+public class HiveAuthzBinding implements AutoCloseable {
   private static final Logger LOG = LoggerFactory
       .getLogger(HiveAuthzBinding.class);
   private static final Splitter ROLE_SET_SPLITTER = Splitter.on(",").trimResults()

http://git-wip-us.apache.org/repos/asf/sentry/blob/3ba5998b/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzPrivileges.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzPrivileges.java b/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzPrivileges.java
index c37ce64..f67510c 100644
--- a/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzPrivileges.java
+++ b/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzPrivileges.java
@@ -20,6 +20,7 @@ import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Map;
 
+import java.util.Objects;
 import org.apache.sentry.core.model.db.DBModelAction;
 import org.apache.sentry.core.model.db.DBModelAuthorizable.AuthorizableType;
 
@@ -138,6 +139,47 @@ public class HiveAuthzPrivileges {
     this.grantOption = requireGrantOption;
   }
 
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof HiveAuthzPrivileges)) {
+      return false;
+    }
+    HiveAuthzPrivileges that = (HiveAuthzPrivileges) o;
+    return grantOption == that.grantOption &&
+      Objects.equals(inputPrivileges, that.inputPrivileges) &&
+      Objects.equals(outputPrivileges, that.outputPrivileges) &&
+      operationType == that.operationType &&
+      operationScope == that.operationScope;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects
+      .hash(inputPrivileges, outputPrivileges, operationType, operationScope, grantOption);
+  }
+
+  /*
+  private boolean equals(Map<AuthorizableType,EnumSet<DBModelAction>> o1, Map<AuthorizableType,EnumSet<DBModelAction>> o2) {
+    if (o1.size() != o2.size()) {
+      return false;
+    }
+
+    for (Map.Entry e1 : o1.entrySet()) {
+      if (!o2.containsKey(e1.getKey())) {
+        return false;
+      }
+
+      if (!o2.get(e1.getKey()).equals(e1.getValue())) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+*/
   /**
    * @return the inputPrivileges
    */

http://git-wip-us.apache.org/repos/asf/sentry/blob/3ba5998b/sentry-binding/sentry-binding-hive-conf/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive-conf/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java b/sentry-binding/sentry-binding-hive-conf/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java
index cd4ae4a..9efd612 100644
--- a/sentry-binding/sentry-binding-hive-conf/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java
+++ b/sentry-binding/sentry-binding-hive-conf/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java
@@ -182,6 +182,13 @@ public class HiveAuthzConf extends Configuration {
         applySystemProperties();
         this.hiveAuthzSiteFile = hiveAuthzSiteURL.toString();
     }
+
+    public HiveAuthzConf() {
+        super();
+        applySystemProperties();
+        this.hiveAuthzSiteFile = null;
+    }
+
     /**
      * Apply system properties to this object if the property name is defined in ConfVars
      * and the value is non-null and not an empty string.

http://git-wip-us.apache.org/repos/asf/sentry/blob/3ba5998b/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 b74516d..db7291f 100644
--- a/sentry-binding/sentry-binding-hive/pom.xml
+++ b/sentry-binding/sentry-binding-hive/pom.xml
@@ -116,6 +116,11 @@ limitations under the License.
       <artifactId>hadoop-minicluster</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.assertj</groupId>
+      <artifactId>assertj-core</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
 </project>

http://git-wip-us.apache.org/repos/asf/sentry/blob/3ba5998b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/DefaultSentryValidator.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/DefaultSentryValidator.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/DefaultSentryValidator.java
index 38ce2db..9de47b3 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/DefaultSentryValidator.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/DefaultSentryValidator.java
@@ -22,8 +22,6 @@ import com.google.common.collect.Sets;
 import java.security.CodeSource;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.EnumSet;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -41,16 +39,14 @@ import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.sentry.binding.hive.SentryOnFailureHookContext;
 import org.apache.sentry.binding.hive.SentryOnFailureHookContextImpl;
 import org.apache.sentry.binding.hive.authz.HiveAuthzBinding.HiveHook;
-import org.apache.sentry.binding.hive.authz.HiveAuthzPrivileges.HiveOperationScope;
+import org.apache.sentry.binding.hive.authz.MetastoreAuthzObjectFilter.ObjectExtractor;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
 import org.apache.sentry.binding.util.SentryAuthorizerUtil;
 import org.apache.sentry.binding.util.SimpleSemanticAnalyzer;
 import org.apache.sentry.core.common.Subject;
 import org.apache.sentry.core.model.db.AccessURI;
 import org.apache.sentry.core.model.db.Column;
-import org.apache.sentry.core.model.db.DBModelAction;
 import org.apache.sentry.core.model.db.DBModelAuthorizable;
-import org.apache.sentry.core.model.db.DBModelAuthorizable.AuthorizableType;
 import org.apache.sentry.core.model.db.Database;
 import org.apache.sentry.core.model.db.Table;
 import org.slf4j.Logger;
@@ -60,6 +56,22 @@ import org.slf4j.LoggerFactory;
  * This class used to do authorization. Check if current user has privileges to do the operation.
  */
 public class DefaultSentryValidator extends SentryHiveAuthorizationValidator {
+  private final MetastoreAuthzObjectFilter.ObjectExtractor<HivePrivilegeObject> OBJECT_EXTRACTOR =
+    new ObjectExtractor<HivePrivilegeObject>() {
+      @Override
+      public String getDatabaseName(HivePrivilegeObject o) {
+        return (o != null) ? o.getDbname() : null;
+      }
+
+      @Override
+      public String getTableName(HivePrivilegeObject o) {
+        return (o != null && isTable(o)) ? o.getObjectName() : null;
+      }
+
+      private boolean isTable(HivePrivilegeObject o) {
+        return o.getType() == HivePrivilegeObjectType.TABLE_OR_VIEW;
+      }
+    };
 
   public static final Logger LOG = LoggerFactory.getLogger(DefaultSentryValidator.class);
 
@@ -356,15 +368,20 @@ public class DefaultSentryValidator extends SentryHiveAuthorizationValidator {
     if (listObjs != null && listObjs.size() >= 1) {
       HivePrivilegeObjectType pType = listObjs.get(0).getType();
       HiveAuthzBinding hiveAuthzBinding = null;
+      MetastoreAuthzObjectFilter authzObjectFilter;
       try {
         switch (pType) {
           case DATABASE:
             hiveAuthzBinding = getAuthzBinding();
-            listObjs = filterShowDatabases(listObjs, authenticator.getUserName(), hiveAuthzBinding);
+            authzObjectFilter = new MetastoreAuthzObjectFilter<HivePrivilegeObject>(hiveAuthzBinding,
+              OBJECT_EXTRACTOR);
+            listObjs = authzObjectFilter.filterDatabases(authenticator.getUserName(), listObjs);
             break;
           case TABLE_OR_VIEW:
             hiveAuthzBinding = getAuthzBinding();
-            listObjs = filterShowTables(listObjs, authenticator.getUserName(), hiveAuthzBinding);
+            authzObjectFilter = new MetastoreAuthzObjectFilter<HivePrivilegeObject>(hiveAuthzBinding,
+              OBJECT_EXTRACTOR);
+            listObjs = authzObjectFilter.filterTables(authenticator.getUserName(), listObjs);
             break;
         }
       } catch (Exception e) {
@@ -393,100 +410,4 @@ public class DefaultSentryValidator extends SentryHiveAuthorizationValidator {
     // false is enough to let Hive know that the query is not required to be transformed.
     return false;
   }
-
-  private List<HivePrivilegeObject> filterShowTables(List<HivePrivilegeObject> listObjs,
-      String userName, HiveAuthzBinding hiveAuthzBinding) {
-    List<HivePrivilegeObject> filteredResult = new ArrayList<HivePrivilegeObject>();
-    Subject subject = new Subject(userName);
-    HiveAuthzPrivileges tableMetaDataPrivilege =
-        new HiveAuthzPrivileges.AuthzPrivilegeBuilder()
-            .addInputObjectPriviledge(AuthorizableType.Column,
-                EnumSet.of(DBModelAction.SELECT, DBModelAction.INSERT, DBModelAction.ALTER,
-                  DBModelAction.DROP, DBModelAction.INDEX, DBModelAction.LOCK))
-            .setOperationScope(HiveOperationScope.TABLE)
-            .setOperationType(
-                HiveAuthzPrivileges.HiveOperationType.INFO)
-            .build();
-
-    for (HivePrivilegeObject obj : listObjs) {
-      // if user has privileges on table, add to filtered list, else discard
-      Table table = new Table(obj.getObjectName());
-      Database database;
-      database = new Database(obj.getDbname());
-
-      Set<List<DBModelAuthorizable>> inputHierarchy = new HashSet<List<DBModelAuthorizable>>();
-      Set<List<DBModelAuthorizable>> outputHierarchy = new HashSet<List<DBModelAuthorizable>>();
-      List<DBModelAuthorizable> externalAuthorizableHierarchy =
-          new ArrayList<DBModelAuthorizable>();
-      externalAuthorizableHierarchy.add(hiveAuthzBinding.getAuthServer());
-      externalAuthorizableHierarchy.add(database);
-      externalAuthorizableHierarchy.add(table);
-      externalAuthorizableHierarchy.add(Column.ALL);
-      inputHierarchy.add(externalAuthorizableHierarchy);
-
-      try {
-        hiveAuthzBinding.authorize(HiveOperation.SHOWTABLES, tableMetaDataPrivilege, subject,
-            inputHierarchy, outputHierarchy);
-        filteredResult.add(obj);
-      } catch (AuthorizationException e) {
-        // squash the exception, user doesn't have privileges, so the table is
-        // not added to
-        // filtered list.
-      }
-    }
-    return filteredResult;
-  }
-
-  private List<HivePrivilegeObject> filterShowDatabases(List<HivePrivilegeObject> listObjs,
-      String userName, HiveAuthzBinding hiveAuthzBinding) {
-    List<HivePrivilegeObject> filteredResult = new ArrayList<HivePrivilegeObject>();
-    Subject subject = new Subject(userName);
-    HiveAuthzPrivileges anyPrivilege =
-        new HiveAuthzPrivileges.AuthzPrivilegeBuilder()
-            .addInputObjectPriviledge(
-                AuthorizableType.Column,
-                EnumSet.of(DBModelAction.SELECT, DBModelAction.INSERT, DBModelAction.ALTER,
-                    DBModelAction.CREATE, DBModelAction.DROP, DBModelAction.INDEX,
-                    DBModelAction.LOCK))
-            .setOperationScope(HiveOperationScope.CONNECT)
-            .setOperationType(
-                HiveAuthzPrivileges.HiveOperationType.QUERY)
-            .build();
-
-    for (HivePrivilegeObject obj : listObjs) {
-      // if user has privileges on database, add to filtered list, else discard
-      Database database = null;
-
-      // if default is not restricted, continue
-      if (DEFAULT_DATABASE_NAME.equalsIgnoreCase(obj.getObjectName())
-          && "false".equalsIgnoreCase(hiveAuthzBinding.getAuthzConf().get(
-              HiveAuthzConf.AuthzConfVars.AUTHZ_RESTRICT_DEFAULT_DB.getVar(), "false"))) {
-        filteredResult.add(obj);
-        continue;
-      }
-
-      database = new Database(obj.getObjectName());
-
-      Set<List<DBModelAuthorizable>> inputHierarchy = new HashSet<List<DBModelAuthorizable>>();
-      Set<List<DBModelAuthorizable>> outputHierarchy = new HashSet<List<DBModelAuthorizable>>();
-      List<DBModelAuthorizable> externalAuthorizableHierarchy =
-          new ArrayList<DBModelAuthorizable>();
-      externalAuthorizableHierarchy.add(hiveAuthzBinding.getAuthServer());
-      externalAuthorizableHierarchy.add(database);
-      externalAuthorizableHierarchy.add(Table.ALL);
-      externalAuthorizableHierarchy.add(Column.ALL);
-      inputHierarchy.add(externalAuthorizableHierarchy);
-
-      try {
-        hiveAuthzBinding.authorize(HiveOperation.SHOWDATABASES, anyPrivilege, subject,
-            inputHierarchy, outputHierarchy);
-        filteredResult.add(obj);
-      } catch (AuthorizationException e) {
-        // squash the exception, user doesn't have privileges, so the table is
-        // not added to
-        // filtered list.
-      }
-    }
-    return filteredResult;
-  }
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/3ba5998b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/MetastoreAuthzObjectFilter.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/MetastoreAuthzObjectFilter.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/MetastoreAuthzObjectFilter.java
new file mode 100644
index 0000000..178780e
--- /dev/null
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/MetastoreAuthzObjectFilter.java
@@ -0,0 +1,189 @@
+/*
+ * 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 static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME;
+
+import com.google.common.collect.Lists;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.List;
+import org.apache.hadoop.hbase.util.Strings;
+import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
+import org.apache.hadoop.hive.ql.plan.HiveOperation;
+import org.apache.sentry.binding.hive.authz.HiveAuthzPrivileges.HiveOperationScope;
+import org.apache.sentry.binding.hive.authz.HiveAuthzPrivileges.HiveOperationType;
+import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
+import org.apache.sentry.core.common.Subject;
+import org.apache.sentry.core.model.db.Column;
+import org.apache.sentry.core.model.db.DBModelAction;
+import org.apache.sentry.core.model.db.DBModelAuthorizable;
+import org.apache.sentry.core.model.db.DBModelAuthorizable.AuthorizableType;
+import org.apache.sentry.core.model.db.Database;
+import org.apache.sentry.core.model.db.Table;
+
+/**
+ * This class uses Sentry authorization to filter a list of HMS metadata objects (or authorization
+ * objects) based on the Sentry privileges that a user is part of. The methods are commonly used
+ * by the Sentry/HMS binding implementations to get a list of objects that a user is allowed to
+ * see.
+ */
+public class MetastoreAuthzObjectFilter<T> {
+  /**
+   * This interface is used to extract information of an object to be filtered.
+   * @param <T>
+   */
+  public interface ObjectExtractor<T> {
+    String getDatabaseName(T t);
+    String getTableName(T t);
+
+  }
+
+  private final HiveAuthzPrivileges LIST_DATABASES_PRIVILEGES = new HiveAuthzPrivileges.AuthzPrivilegeBuilder()
+    .addInputObjectPriviledge(
+      AuthorizableType.Column,
+      EnumSet.of(
+        DBModelAction.SELECT, DBModelAction.INSERT, DBModelAction.ALTER,
+        DBModelAction.CREATE, DBModelAction.DROP, DBModelAction.INDEX,
+        DBModelAction.LOCK))
+    .addInputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.SELECT))
+    .setOperationScope(HiveOperationScope.CONNECT)
+    .setOperationType(HiveOperationType.QUERY)
+    .build();
+
+  private final HiveAuthzPrivileges LIST_TABLES_PRIVILEGES = new HiveAuthzPrivileges.AuthzPrivilegeBuilder()
+    .addInputObjectPriviledge(
+      AuthorizableType.Column,
+      EnumSet.of(
+        DBModelAction.SELECT, DBModelAction.INSERT, DBModelAction.ALTER,
+        DBModelAction.DROP, DBModelAction.INDEX, DBModelAction.LOCK))
+    .setOperationScope(HiveOperationScope.TABLE)
+    .setOperationType(
+      HiveAuthzPrivileges.HiveOperationType.INFO)
+    .build();
+
+  private final boolean DEFAULT_DATABASE_RESTRICTED;
+  private final DBModelAuthorizable AUTH_SERVER;
+  private HiveAuthzBinding authzBinding;
+  private ObjectExtractor extractor;
+
+  public MetastoreAuthzObjectFilter(HiveAuthzBinding authzBinding, ObjectExtractor extractor) {
+    this.authzBinding = authzBinding;
+    this.extractor = extractor;
+    this.AUTH_SERVER = authzBinding.getAuthServer();
+    this.DEFAULT_DATABASE_RESTRICTED = authzBinding.getAuthzConf()
+      .getBoolean(HiveAuthzConf.AuthzConfVars.AUTHZ_RESTRICT_DEFAULT_DB.getVar(), false);
+  }
+
+  /**
+   * Filter a list of {@code dbNames} objects based on the authorization privileges of {@code subject}.
+   *
+   * @param username The username to request authorization from.
+   * @param dbNames A list of databases that must be filtered based on the user privileges.
+   * @return A list of database objects filtered by the privileges of the user. If a null value is
+   * passed as {@code dbNames}, then an empty list is returned.
+   */
+  public List<T> filterDatabases(String username, List<T> dbNames) {
+    if (dbNames == null) {
+      return Collections.emptyList();
+    }
+
+    List<T> filteredDatabases = Lists.newArrayList();
+    for (T dbName : dbNames) {
+      String objName = extractor.getDatabaseName(dbName);
+      if (Strings.isEmpty(objName) || authorizeDatabase(username, objName)) {
+        filteredDatabases.add(dbName);
+      }
+    }
+
+    return filteredDatabases;
+  }
+
+  /**
+   * Filter a list of {@code tableNames} objects based on the authorization privileges of {@code subject}.
+   *
+   * @param username The username to request authorization from.
+   * @param tables A list of tables that must be filtered based on the user privileges.
+   * @return A list of tables objects filtered by the privileges of the user. If a null value is
+   * passed as {@code tableNames}, then an empty list is returned.
+   */
+  public List<T> filterTables(String username, List<T> tables) {
+    if (tables == null) {
+      return Collections.emptyList();
+    }
+
+    List<T> filteredTables = Lists.newArrayList();
+    for (T table : tables) {
+      String dbName = extractor.getDatabaseName(table);
+      String tableName = extractor.getTableName(table);
+      if (Strings.isEmpty(dbName) || authorizeTable(username, dbName, tableName)) {
+        filteredTables.add(table);
+      }
+    }
+
+    return filteredTables;
+  }
+
+  /**
+   * Checks if a database is authorized to be accessed by the specific user.
+   * @return True if it is authorized, false otherwise.
+   */
+  private boolean authorizeDatabase(String username, String dbName) {
+    if (!DEFAULT_DATABASE_RESTRICTED && dbName.equalsIgnoreCase(DEFAULT_DATABASE_NAME)) {
+      return true;
+    }
+
+    Database database = new Database(dbName);
+    List<DBModelAuthorizable> authorizable = Arrays.asList(
+      AUTH_SERVER, database, Table.ALL, Column.ALL
+    );
+
+    return authorize(HiveOperation.SHOWDATABASES, LIST_DATABASES_PRIVILEGES, username, authorizable);
+  }
+
+  /**
+   * Checks if a table is authorized to be accessed by the specific user.
+   * @return True if it is authorized, false otherwise.
+   */
+  private boolean authorizeTable(String username, String dbName, String tableName) {
+    Database database = new Database(dbName);
+    Table table = new Table(tableName);
+
+    List<DBModelAuthorizable> authorizable = Arrays.asList(
+      AUTH_SERVER, database, table, Column.ALL
+    );
+
+    return authorize(HiveOperation.SHOWTABLES, LIST_TABLES_PRIVILEGES, username, authorizable);
+  }
+
+  /**
+   * Calls the authorization method of Sentry to check the access to a specific authorizable.
+   * @return True if it is authorized, false otherwise.
+   */
+  private boolean authorize(HiveOperation op, HiveAuthzPrivileges privs, String username, List<DBModelAuthorizable> authorizable) {
+    try {
+      authzBinding.authorize(op, privs, new Subject(username),
+        Collections.singleton(authorizable), Collections.emptySet());
+    } catch (AuthorizationException e) {
+      return false;
+    }
+
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/sentry/blob/3ba5998b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/AuthorizingObjectStore.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/AuthorizingObjectStore.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/AuthorizingObjectStore.java
index 92eb136..1399150 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/AuthorizingObjectStore.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/AuthorizingObjectStore.java
@@ -37,11 +37,10 @@ import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
 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.shims.Utils;
-import org.apache.sentry.binding.hive.authz.HiveAuthzBindingHookBase;
 import org.apache.sentry.binding.hive.authz.HiveAuthzBinding;
+import org.apache.sentry.binding.hive.authz.MetastoreAuthzObjectFilter;
+import org.apache.sentry.binding.hive.authz.MetastoreAuthzObjectFilter.ObjectExtractor;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars;
 
@@ -285,9 +284,21 @@ public class AuthorizingObjectStore extends ObjectStore {
       throws MetaException {
     if (needsAuthorization(getUserName())) {
       try {
-        return HiveAuthzBindingHookBase.filterShowDatabases(getHiveAuthzBinding(),
-            dbList, HiveOperation.SHOWDATABASES, getUserName());
-      } catch (SemanticException e) {
+        MetastoreAuthzObjectFilter<String> filter = new MetastoreAuthzObjectFilter<>(
+          getHiveAuthzBinding(), new ObjectExtractor<String>() {
+          @Override
+          public String getDatabaseName(String o) {
+            return o;
+          }
+
+          @Override
+          public String getTableName(String o) {
+            return null;
+          }
+        });
+
+        return filter.filterDatabases(getUserName(), dbList);
+      } catch (Exception e) {
         throw new MetaException("Error getting DB list " + e.getMessage());
       }
     } else {
@@ -306,9 +317,21 @@ public class AuthorizingObjectStore extends ObjectStore {
       throws MetaException {
     if (needsAuthorization(getUserName())) {
       try {
-        return HiveAuthzBindingHookBase.filterShowTables(getHiveAuthzBinding(),
-            tabList, HiveOperation.SHOWTABLES, getUserName(), dbName);
-      } catch (SemanticException e) {
+        MetastoreAuthzObjectFilter<String> filter = new MetastoreAuthzObjectFilter<>(
+          getHiveAuthzBinding(), new ObjectExtractor<String>() {
+          @Override
+          public String getDatabaseName(String o) {
+            return dbName;
+          }
+
+          @Override
+          public String getTableName(String o) {
+            return o;
+          }
+        });
+
+        return filter.filterTables(getUserName(), tabList);
+      } catch (Exception e) {
         throw new MetaException("Error getting Table list " + e.getMessage());
       }
     } else {
@@ -391,7 +414,8 @@ public class AuthorizingObjectStore extends ObjectStore {
    * @return
    */
   private boolean needsAuthorization(String userName) throws MetaException {
-    return !getServiceUsers().contains(userName.trim());
+    // Username should be case sensitive
+    return !getServiceUsers().contains(userName);
   }
 
   private static Set<String> toTrimed(Set<String> s) {

http://git-wip-us.apache.org/repos/asf/sentry/blob/3ba5998b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/AuthorizingObjectStoreBase.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/AuthorizingObjectStoreBase.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/AuthorizingObjectStoreBase.java
index d015085..c4c66f3 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/AuthorizingObjectStoreBase.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/AuthorizingObjectStoreBase.java
@@ -32,11 +32,10 @@ import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
 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.shims.Utils;
-import org.apache.sentry.binding.hive.authz.HiveAuthzBindingHookBase;
 import org.apache.sentry.binding.hive.authz.HiveAuthzBinding;
+import org.apache.sentry.binding.hive.authz.MetastoreAuthzObjectFilter;
+import org.apache.sentry.binding.hive.authz.MetastoreAuthzObjectFilter.ObjectExtractor;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars;
 
@@ -283,9 +282,21 @@ public class AuthorizingObjectStoreBase extends ObjectStore {
       throws MetaException {
     if (needsAuthorization(getUserName())) {
       try {
-        return HiveAuthzBindingHookBase.filterShowDatabases(getHiveAuthzBinding(),
-            dbList, HiveOperation.SHOWDATABASES, getUserName());
-      } catch (SemanticException e) {
+        MetastoreAuthzObjectFilter<String> filter = new MetastoreAuthzObjectFilter<>(
+          getHiveAuthzBinding(), new ObjectExtractor<String>() {
+          @Override
+          public String getDatabaseName(String o) {
+            return o;
+          }
+
+          @Override
+          public String getTableName(String o) {
+            return null;
+          }
+        });
+
+        return filter.filterDatabases(getUserName(), dbList);
+      } catch (Exception e) {
         throw new MetaException("Error getting DB list " + e.getMessage());
       }
     } else {
@@ -304,9 +315,21 @@ public class AuthorizingObjectStoreBase extends ObjectStore {
       throws MetaException {
     if (needsAuthorization(getUserName())) {
       try {
-        return HiveAuthzBindingHookBase.filterShowTables(getHiveAuthzBinding(),
-            tabList, HiveOperation.SHOWTABLES, getUserName(), dbName);
-      } catch (SemanticException e) {
+        MetastoreAuthzObjectFilter<String> filter = new MetastoreAuthzObjectFilter<>(
+          getHiveAuthzBinding(), new ObjectExtractor<String>() {
+          @Override
+          public String getDatabaseName(String o) {
+            return dbName;
+          }
+
+          @Override
+          public String getTableName(String o) {
+            return o;
+          }
+        });
+
+        return filter.filterTables(getUserName(), tabList);
+      } catch (Exception e) {
         throw new MetaException("Error getting Table list " + e.getMessage());
       }
     } else {
@@ -389,7 +412,8 @@ public class AuthorizingObjectStoreBase extends ObjectStore {
    * @return
    */
   private boolean needsAuthorization(String userName) throws MetaException {
-    return !getServiceUsers().contains(userName.trim());
+    // Username is case sensitive
+    return !getServiceUsers().contains(userName);
   }
 
   private static Set<String> toTrimed(Set<String> s) {

http://git-wip-us.apache.org/repos/asf/sentry/blob/3ba5998b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/HiveAuthzBindingFactory.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/HiveAuthzBindingFactory.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/HiveAuthzBindingFactory.java
new file mode 100644
index 0000000..e516fe1
--- /dev/null
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/HiveAuthzBindingFactory.java
@@ -0,0 +1,30 @@
+/**
+ * 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 org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.sentry.binding.hive.authz.HiveAuthzBinding;
+import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
+
+/**
+ * Factory class that creates a new HiveAuthzBinding.
+ */
+public interface HiveAuthzBindingFactory {
+  HiveAuthzBinding fromMetaStoreConf(HiveConf hiveConf, HiveAuthzConf authzConf) throws Exception;
+  String getUserName();
+}

http://git-wip-us.apache.org/repos/asf/sentry/blob/3ba5998b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/MetastoreAuthzBindingBase.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/MetastoreAuthzBindingBase.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/MetastoreAuthzBindingBase.java
index 8ad9e50..328d2b5 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/MetastoreAuthzBindingBase.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/MetastoreAuthzBindingBase.java
@@ -416,7 +416,7 @@ public abstract class MetastoreAuthzBindingBase extends MetaStorePreEventListene
     return !serviceUsers.contains(userName);
   }
 
-  private static Set<String> toTrimedLower(Set<String> s) {
+  public static Set<String> toTrimedLower(Set<String> s) {
     Set<String> result = Sets.newHashSet();
     for (String v : s) {
       result.add(v.trim().toLowerCase());

http://git-wip-us.apache.org/repos/asf/sentry/blob/3ba5998b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/SentryHiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/SentryHiveMetaStoreClient.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/SentryHiveMetaStoreClient.java
index e30a860..b77a81d 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/SentryHiveMetaStoreClient.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/SentryHiveMetaStoreClient.java
@@ -25,11 +25,10 @@ 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.authz.HiveAuthzBindingHookBase;
 import org.apache.sentry.binding.hive.authz.HiveAuthzBinding;
+import org.apache.sentry.binding.hive.authz.MetastoreAuthzObjectFilter;
+import org.apache.sentry.binding.hive.authz.MetastoreAuthzObjectFilter.ObjectExtractor;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
 import org.apache.thrift.TException;
 
@@ -92,9 +91,21 @@ public class SentryHiveMetaStoreClient extends HiveMetaStoreClient implements
   private List<String> filterDatabases(List<String> dbList)
       throws MetaException {
     try {
-      return HiveAuthzBindingHookBase.filterShowDatabases(getHiveAuthzBinding(),
-          dbList, HiveOperation.SHOWDATABASES, getUserName());
-    } catch (SemanticException e) {
+      MetastoreAuthzObjectFilter<String> filter = new MetastoreAuthzObjectFilter<>(
+        getHiveAuthzBinding(), new ObjectExtractor<String>() {
+        @Override
+        public String getDatabaseName(String o) {
+          return o;
+        }
+
+        @Override
+        public String getTableName(String o) {
+          return null;
+        }
+      });
+
+      return filter.filterDatabases(getUserName(), dbList);
+    } catch (Exception e) {
       throw new MetaException("Error getting DB list " + e.getMessage());
     }
   }
@@ -110,9 +121,21 @@ public class SentryHiveMetaStoreClient extends HiveMetaStoreClient implements
   private List<String> filterTables(String dbName, List<String> tabList)
       throws MetaException {
     try {
-      return HiveAuthzBindingHookBase.filterShowTables(getHiveAuthzBinding(),
-          tabList, HiveOperation.SHOWTABLES, getUserName(), dbName);
-    } catch (SemanticException e) {
+      MetastoreAuthzObjectFilter<String> filter = new MetastoreAuthzObjectFilter<>(
+        getHiveAuthzBinding(), new ObjectExtractor<String>() {
+        @Override
+        public String getDatabaseName(String o) {
+          return dbName;
+        }
+
+        @Override
+        public String getTableName(String o) {
+          return o;
+        }
+      });
+
+      return filter.filterTables(getUserName(), tabList);
+    } catch (Exception e) {
       throw new MetaException("Error getting Table list " + e.getMessage());
     }
   }

http://git-wip-us.apache.org/repos/asf/sentry/blob/3ba5998b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/SentryMetaStoreFilterHook.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/SentryMetaStoreFilterHook.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/SentryMetaStoreFilterHook.java
index 5ecc87f..312c5db 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/SentryMetaStoreFilterHook.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/SentryMetaStoreFilterHook.java
@@ -17,6 +17,9 @@
  */
 package org.apache.sentry.binding.metastore;
 
+import com.google.common.collect.Sets;
+import java.util.Collections;
+import java.util.Set;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -28,25 +31,86 @@ 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.authz.HiveAuthzBindingHookBase;
+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.authz.HiveAuthzBinding.HiveHook;
+import org.apache.sentry.binding.hive.authz.MetastoreAuthzObjectFilter;
 
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.util.ArrayList;
 import java.util.List;
+import org.apache.sentry.binding.hive.authz.MetastoreAuthzObjectFilter.ObjectExtractor;
+import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
 
+/**
+ * {@code} SentryMetaStoreFilterHook} may be used by the HMS server to filter databases, tables
+ * and partitions that are authorized to be seen by a user making the HMS request. Usage on the
+ * {@link org.apache.hadoop.hive.metastore.HiveMetaStoreClient} was dropped when Hive authz V2 was
+ * added in Hive v2.x and higher.
+ *
+ * <p/>
+ * Connections to HMS are usually done as admins (or any of the Sentry service users), so this
+ * class will not filter anything; but others component, such as Spark, can commonly make
+ * this requests as a normal user, which require a proper authorization to to return only those
+ * objects that the user is able to see.
+ */
 public class SentryMetaStoreFilterHook implements MetaStoreFilterHook {
-
   static final protected Log LOG = LogFactory.getLog(SentryMetaStoreFilterHook.class);
 
+  private final HiveConf hiveConf;
   private HiveAuthzBinding hiveAuthzBinding;
+  private HiveAuthzBindingFactory authzBindingFactory;
   private HiveAuthzConf authzConf;
+  private Set<String> serviceUsers;
+
+  /**
+   * Instatiates a new {@code SentryMetaStoreFilterHook} object with a default
+   * {@code HiveAuthzBindingFactory} implementation that calls the Sentry server to filter
+   * the Metastore objects.
+   *
+   * @param hiveConf The HiveConf object that contains configuration to connect to Sentry.
+   */
+  public SentryMetaStoreFilterHook(HiveConf hiveConf) {
+    this(hiveConf, HiveAuthzConf.getAuthzConf(hiveConf), new HiveAuthzBindingFactory() {
+      @Override
+      public HiveAuthzBinding fromMetaStoreConf(HiveConf hiveConf, HiveAuthzConf authzConf) throws Exception {
+        return new HiveAuthzBinding(HiveHook.HiveMetaStore, hiveConf, authzConf);
+      }
+
+      @Override
+      public String getUserName() {
+        try {
+          /*
+           * Returns the HMS username by looking intto the UGI class. This is called during
+           * the filtering calls (not in the constructor) because HMS may do these requests
+           * with different users set in the UGI.
+           */
+          return Utils.getUGI().getShortUserName();
+        } catch (Exception e) {
+          throw new RuntimeException("Unable to get the HMS username: " + e.getMessage());
+        }
+      }
+    });
+  }
 
-  public SentryMetaStoreFilterHook(HiveConf hiveConf) { //NOPMD
+  /**
+   * Instatiates a new {@code SentryMetaStoreFilterHook} object with a specific
+   * {@code HiveAuthzBindingFactory} implementation to get authorization to filter the Metastore
+   * objects.
+   *
+   * @param hiveConf The HiveConf object that contains configuration to connect to Sentry.
+   * @param authzConf The HiveAuthzConf object that contains Sentry settings.
+   * @param authzBindingFactory An implementation to get the sentry/hive binding object to get
+   * authorization to filter the Metastore objects.
+   */
+  public SentryMetaStoreFilterHook(HiveConf hiveConf, HiveAuthzConf authzConf, HiveAuthzBindingFactory authzBindingFactory) {
+    this.hiveConf = hiveConf;
+    this.authzConf = authzConf;
+    this.authzBindingFactory = authzBindingFactory;
+
+    // Users must be case sensitive to be compatible with Hadoop and Unix user names.
+    this.serviceUsers = Sets.newHashSet(authzConf.getTrimmedStringCollection(
+      HiveAuthzConf.AuthzConfVars.AUTHZ_METASTORE_SERVICE_USERS.getVar()));
+
+    LOG.info("SentryMetaStoreFilterHook initialized with service users: " + this.serviceUsers);
   }
 
   @Override
@@ -57,6 +121,11 @@ public class SentryMetaStoreFilterHook implements MetaStoreFilterHook {
   @Override
   public Database filterDatabase(Database dataBase)
       throws NoSuchObjectException {
+    String name = dataBase.getName();
+    if (filterDb(Collections.singletonList(name)).isEmpty()) {
+      throw new NoSuchObjectException(String.format("Database %s does not exist", name));
+    }
+
     return dataBase;
   }
 
@@ -67,48 +136,62 @@ public class SentryMetaStoreFilterHook implements MetaStoreFilterHook {
 
   @Override
   public Table filterTable(Table table) throws NoSuchObjectException {
+    String dbName = table.getDbName();
+    String tableName = table.getTableName();
+
+    if (filterTab(dbName, Collections.singletonList(tableName)).isEmpty()) {
+      throw new NoSuchObjectException(String.format("Table %s.%s does not exist", dbName, tableName));
+    }
+
     return table;
   }
 
   @Override
   public List<Table> filterTables(List<Table> tableList) {
-    return tableList;
+    return filterTab(tableList);
   }
 
+  // Sentry does not support partition filtering
   @Override
   public List<Partition> filterPartitions(List<Partition> partitionList) {
     return partitionList;
   }
 
+  // Sentry does not support partition filtering
   @Override
   public List<PartitionSpec> filterPartitionSpecs(
       List<PartitionSpec> partitionSpecList) {
     return partitionSpecList;
   }
 
+  // Sentry does not support partition filtering
   @Override
   public Partition filterPartition(Partition partition)
       throws NoSuchObjectException {
     return partition;
   }
 
+  // Sentry does not support partition filtering
   @Override
   public List<String> filterPartitionNames(String dbName, String tblName,
       List<String> partitionNames) {
     return partitionNames;
   }
 
+  // Sentry does not support index filtering
   @Override
   public Index filterIndex(Index index) throws NoSuchObjectException {
     return index;
   }
 
+  // Sentry does not support index filtering
   @Override
   public List<String> filterIndexNames(String dbName, String tblName,
       List<String> indexList) {
     return indexList;
   }
 
+  // Sentry does not support index filtering
   @Override
   public List<Index> filterIndexes(List<Index> indexeList) {
     return indexeList;
@@ -122,14 +205,30 @@ public class SentryMetaStoreFilterHook implements MetaStoreFilterHook {
    * @throws MetaException
    */
   private List<String> filterDb(List<String> dbList) {
-    try {
-      return HiveAuthzBindingHookBase.filterShowDatabases(getHiveAuthzBinding(),
-          dbList, HiveOperation.SHOWDATABASES, getUserName());
+    // If the user is part of the Sentry service user list, then skip the authorization and
+    // do not filter the objects.
+    if (!needsAuthorization(authzBindingFactory.getUserName())) {
+      return dbList;
+    }
+
+    try (HiveAuthzBinding authzBinding = getHiveAuthzBinding()) {
+      MetastoreAuthzObjectFilter<String> filter = new MetastoreAuthzObjectFilter<>(authzBinding,
+        new ObjectExtractor<String>() {
+          @Override
+          public String getDatabaseName(String o) {
+            return o;
+          }
+
+          @Override
+          public String getTableName(String s) {
+            return null;
+          }
+        });
+
+      return filter.filterDatabases(authzBindingFactory.getUserName(), dbList);
     } catch (Exception e) {
       LOG.warn("Error getting DB list ", e);
-      return new ArrayList<String>();
-    } finally {
-      close();
+      return Collections.emptyList();
     }
   }
 
@@ -141,19 +240,66 @@ public class SentryMetaStoreFilterHook implements MetaStoreFilterHook {
    * @throws MetaException
    */
   private List<String> filterTab(String dbName, List<String> tabList) {
-    try {
-      return HiveAuthzBindingHookBase.filterShowTables(getHiveAuthzBinding(),
-          tabList, HiveOperation.SHOWTABLES, getUserName(), dbName);
+    // If the user is part of the Sentry service user list, then skip the authorization and
+    // do not filter the objects.
+    if (!needsAuthorization(authzBindingFactory.getUserName())) {
+      return tabList;
+    }
+
+    try (HiveAuthzBinding authzBinding = getHiveAuthzBinding()) {
+      MetastoreAuthzObjectFilter<String> filter = new MetastoreAuthzObjectFilter<>(authzBinding,
+        new ObjectExtractor<String>() {
+          @Override
+          public String getDatabaseName(String o) {
+            return dbName;
+          }
+
+          @Override
+          public String getTableName(String o) {
+            return o;
+          }
+        });
+
+      return filter.filterTables(authzBindingFactory.getUserName(), tabList);
     } catch (Exception e) {
       LOG.warn("Error getting Table list ", e);
-      return new ArrayList<String>();
-    } finally {
-      close();
+      return Collections.emptyList();
     }
   }
 
-  private String getUserName() {
-    return getConf().get(HiveAuthzConf.HIVE_SENTRY_SUBJECT_NAME);
+  /**
+   * Invoke Hive table filtering that removes the entries which use has no
+   * privileges to access
+   * @param tabList
+   * @return
+   * @throws MetaException
+   */
+  private List<Table> filterTab(List<Table> tabList) {
+    // If the user is part of the Sentry service user list, then skip the authorization and
+    // do not filter the objects.
+    if (!needsAuthorization(authzBindingFactory.getUserName())) {
+      return tabList;
+    }
+
+    try (HiveAuthzBinding authzBinding = getHiveAuthzBinding()) {
+      MetastoreAuthzObjectFilter<Table> filter = new MetastoreAuthzObjectFilter<>(authzBinding,
+        new ObjectExtractor<Table>() {
+          @Override
+          public String getDatabaseName(Table o) {
+            return (o != null) ? o.getDbName() : null;
+          }
+
+          @Override
+          public String getTableName(Table o) {
+            return (o != null) ? o.getTableName() : null;
+          }
+        });
+
+      return filter.filterTables(authzBindingFactory.getUserName(), tabList);
+    } catch (Exception e) {
+      LOG.warn("Error getting Table list ", e);
+      return Collections.emptyList();
+    }
   }
 
   /**
@@ -163,39 +309,18 @@ public class SentryMetaStoreFilterHook implements MetaStoreFilterHook {
    */
   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);
+        hiveAuthzBinding = authzBindingFactory.fromMetaStoreConf(hiveConf, authzConf);
       } catch (Exception e) {
-        throw new MetaException("Failed to load Hive binding " + e.getMessage());
+        throw new MetaException("The Sentry/Hive authz binding could not be created: "
+          + e.getMessage());
       }
     }
-    return hiveAuthzBinding;
-  }
 
-  private HiveConf getConf() {
-    return SessionState.get().getConf();
+    return hiveAuthzBinding;
   }
 
-  private void close() {
-    if (hiveAuthzBinding != null) {
-      hiveAuthzBinding.close();
-      hiveAuthzBinding = null;
-    }
+  private boolean needsAuthorization(String username) {
+    return !serviceUsers.contains(username);
   }
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/3ba5998b/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/authz/TestMetastoreAuthzObjectFilter.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/authz/TestMetastoreAuthzObjectFilter.java b/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/authz/TestMetastoreAuthzObjectFilter.java
new file mode 100644
index 0000000..3ca89be
--- /dev/null
+++ b/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/authz/TestMetastoreAuthzObjectFilter.java
@@ -0,0 +1,323 @@
+/*
+ * 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 static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.List;
+import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
+import org.apache.hadoop.hive.ql.plan.HiveOperation;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType;
+import org.apache.sentry.binding.hive.authz.HiveAuthzPrivileges.HiveOperationScope;
+import org.apache.sentry.binding.hive.authz.HiveAuthzPrivileges.HiveOperationType;
+import org.apache.sentry.binding.hive.authz.MetastoreAuthzObjectFilter.ObjectExtractor;
+import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
+import org.apache.sentry.core.common.Subject;
+import org.apache.sentry.core.model.db.Column;
+import org.apache.sentry.core.model.db.DBModelAction;
+import org.apache.sentry.core.model.db.DBModelAuthorizable;
+import org.apache.sentry.core.model.db.DBModelAuthorizable.AuthorizableType;
+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.assertj.core.api.iterable.Extractor;
+import org.assertj.core.util.Lists;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class TestMetastoreAuthzObjectFilter {
+  // Mock the HiveAuthzBinding to avoid making real connections to a Sentry server
+  private HiveAuthzBinding mockBinding = Mockito.mock(HiveAuthzBinding.class);
+
+  private HiveAuthzConf authzConf = new HiveAuthzConf();
+  private final Server SERVER1 = new Server("server1");
+
+  private final MetastoreAuthzObjectFilter.ObjectExtractor<String> DB_NAME_EXTRACTOR =
+    new ObjectExtractor<String>() {
+      @Override
+      public String getDatabaseName(String s) {
+        return s;
+      }
+
+      @Override
+      public String getTableName(String s) {
+        return null;
+      }
+    };
+
+  private final MetastoreAuthzObjectFilter.ObjectExtractor<HivePrivilegeObject> HIVE_OBJECT_EXTRACTOR =
+    new ObjectExtractor<HivePrivilegeObject>() {
+      @Override
+      public String getDatabaseName(HivePrivilegeObject o) {
+        return (o != null) ? o.getDbname() : null;
+      }
+
+      @Override
+      public String getTableName(HivePrivilegeObject o) {
+        return (o != null) ? o.getObjectName() : null;
+      }
+    };
+
+  private final HiveAuthzPrivileges LIST_DATABASES_PRIVILEGES = new HiveAuthzPrivileges.AuthzPrivilegeBuilder()
+    .addInputObjectPriviledge(
+      AuthorizableType.Column,
+      EnumSet.of(
+        DBModelAction.SELECT, DBModelAction.INSERT, DBModelAction.ALTER,
+        DBModelAction.CREATE, DBModelAction.DROP, DBModelAction.INDEX,
+        DBModelAction.LOCK))
+    .addInputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.SELECT))
+    .setOperationScope(HiveOperationScope.CONNECT)
+    .setOperationType(HiveOperationType.QUERY)
+    .build();
+
+  private final HiveAuthzPrivileges LIST_TABLES_PRIVILEGES = new HiveAuthzPrivileges.AuthzPrivilegeBuilder()
+    .addInputObjectPriviledge(
+      AuthorizableType.Column,
+      EnumSet.of(
+        DBModelAction.SELECT, DBModelAction.INSERT, DBModelAction.ALTER,
+        DBModelAction.DROP, DBModelAction.INDEX, DBModelAction.LOCK))
+    .setOperationScope(HiveOperationScope.TABLE)
+    .setOperationType(
+      HiveAuthzPrivileges.HiveOperationType.INFO)
+    .build();
+
+  /**
+   * Internal class used by AssertJ extract() method to extract the object name of
+   * a HivePrivilegeObject
+   */
+  static class HiveObjectExtractor implements Extractor<HivePrivilegeObject, String> {
+    private HiveObjectExtractor() {}
+
+    public static Extractor<HivePrivilegeObject, String> objectName() {
+      return new HiveObjectExtractor();
+    }
+
+    @Override
+    public String extract(HivePrivilegeObject input) {
+      return input.getObjectName();
+    }
+  }
+
+  private void restrictDefaultDatabase(boolean b) {
+    authzConf.setBoolean(HiveAuthzConf.AuthzConfVars.AUTHZ_RESTRICT_DEFAULT_DB.getVar(), b);
+  }
+
+  private MetastoreAuthzObjectFilter.ObjectExtractor<String> createTableStringExtractor(String dbName) {
+    return new ObjectExtractor<String>() {
+      @Override
+      public String getDatabaseName(String s) {
+        return dbName;
+      }
+
+      @Override
+      public String getTableName(String s) {
+        return s;
+      }
+    };
+  }
+
+  // Mock the authorize() method to throw an exception for the following list of databases
+  private void restrictDatabaseNamesOnBinding(String username, List<String> dbNames) {
+    for (String dbName : dbNames) {
+      Database database = new Database(dbName);
+      List<DBModelAuthorizable> authorizable = Arrays.asList(
+        SERVER1, database, Table.ALL, Column.ALL
+      );
+
+      Mockito.doThrow(new AuthorizationException())
+        .when(mockBinding).authorize(HiveOperation.SHOWDATABASES,
+        LIST_DATABASES_PRIVILEGES, new Subject(username), Collections.singleton(authorizable), Collections.emptySet());
+    }
+  }
+
+  // Mock the authorize() method to throw an exception for the following list of databases
+  private void restrictTablesNamesOnBinding(String username, String dbName, List<String> tableNames) {
+    Database database = new Database(dbName);
+
+    for (String tableName : tableNames) {
+      Table table = new Table(tableName);
+      List<DBModelAuthorizable> authorizable = Arrays.asList(
+        SERVER1, database, table, Column.ALL
+      );
+
+      Mockito.doThrow(new AuthorizationException())
+        .when(mockBinding).authorize(HiveOperation.SHOWTABLES,
+        LIST_TABLES_PRIVILEGES, new Subject(username), Collections.singleton(authorizable), Collections.emptySet());
+    }
+  }
+
+  // Converts a list of database HivePrivilegeObject to a HivePrivilegeObject list
+  private List<HivePrivilegeObject> createHivePrivilegeDatabaseList(String ... dbNames) {
+    List<HivePrivilegeObject> hiveObjects = Lists.newArrayList();
+    for (String dbName : dbNames) {
+      hiveObjects.add(new HivePrivilegeObject(HivePrivilegeObjectType.DATABASE, dbName, dbName));
+    }
+
+    return hiveObjects;
+  }
+
+  // Converts a list of table HivePrivilegeObject to a HivePrivilegeObject list
+  private List<HivePrivilegeObject> createHivePrivilegeTableList(String dbName, String ... tableNames) {
+    List<HivePrivilegeObject> hiveObjects = Lists.newArrayList();
+    for (String tableName : tableNames) {
+      hiveObjects.add(new HivePrivilegeObject(HivePrivilegeObjectType.DATABASE, dbName, tableName));
+    }
+
+    return hiveObjects;
+  }
+
+  @Before
+  public void setup() {
+    // Reset the mocks in case it was modified on the test methods
+    Mockito.reset(mockBinding);
+
+    Mockito.when(mockBinding.getAuthServer()).thenReturn(SERVER1);
+    Mockito.when(mockBinding.getAuthzConf()).thenReturn(authzConf);
+
+    // Do not restrict the 'default' database by default
+    restrictDefaultDatabase(false);
+  }
+
+  @Test
+  public void testFilterDatabaseStrings() {
+    final String USER1 = "user1";
+
+    MetastoreAuthzObjectFilter filter = new MetastoreAuthzObjectFilter(mockBinding, DB_NAME_EXTRACTOR);
+    assertThat(filter).isNotNull();
+
+    // Verify that null or empty lists do return an empty list (not null values to avoid NPE)
+    assertThat(filter.filterDatabases(USER1, null)).isNotNull().isEmpty();
+    assertThat(filter.filterDatabases(USER1, Collections.emptyList())).isNotNull().isEmpty();
+
+    // Verify that null or empty items in the list are not filtered out and not cause exceptions
+    assertThat(filter.filterDatabases(USER1, Arrays.asList(null, "", null, null, "")))
+      .containsExactly(null, "", null, null, "");
+
+    // Verify restricted databases db1,db3,db5 are filtered out
+    restrictDatabaseNamesOnBinding(USER1, Arrays.asList("db1", "db3", "db5"));
+    assertThat(filter.filterDatabases(USER1, Arrays.asList("db1", "db2", "db3", "db4", "db5")))
+      .containsExactly("db2", "db4");
+  }
+
+  @Test
+  public void testFilterDatabaseHiveObjects() {
+    final String USER1 = "user1";
+
+    MetastoreAuthzObjectFilter filter =
+      new MetastoreAuthzObjectFilter(mockBinding, HIVE_OBJECT_EXTRACTOR);
+    assertThat(filter).isNotNull();
+
+    // Verify that null or empty lists do return an empty list (not null values to avoid NPE)
+    assertThat(filter.filterDatabases(USER1, null)).isNotNull().isEmpty();
+    assertThat(filter.filterDatabases(USER1, Collections.emptyList())).isNotNull().isEmpty();
+
+    // Verify that null or empty items in the list are not filtered out and not cause exceptions
+    assertThat(filter.filterDatabases(USER1, Arrays.asList(null, null)))
+      .containsExactly(null, null);
+
+    // Verify restricted databases db1,db3,db5 are filtered out
+    restrictDatabaseNamesOnBinding(USER1, Arrays.asList("db1", "db3", "db5"));
+    assertThat(filter.filterDatabases(USER1, createHivePrivilegeDatabaseList("db1", "db2", "db3", "db4", "db5")))
+      .extracting(HiveObjectExtractor.objectName()).containsExactly("db2", "db4");
+  }
+
+  @Test
+  public void testFilterDatabasesRestrictDefaultDatabase() {
+    final String USER1 = "user1";
+    MetastoreAuthzObjectFilter filter;
+
+    restrictDatabaseNamesOnBinding(USER1, Arrays.asList("db1", "default"));
+
+    // Verify the default database is restricted when filtering strings
+    restrictDefaultDatabase(true);
+    filter = new MetastoreAuthzObjectFilter<String>(mockBinding, DB_NAME_EXTRACTOR);
+    assertThat(filter.filterDatabases(USER1, Arrays.asList("db1", "default", "db4")))
+      .containsExactly("db4");
+
+    // Verify the default database is not restricted when filtering strings
+    restrictDefaultDatabase(false);
+    filter = new MetastoreAuthzObjectFilter<String>(mockBinding, DB_NAME_EXTRACTOR);
+    assertThat(filter.filterDatabases(USER1, Arrays.asList("db1", "default", "db4")))
+      .containsExactly("default", "db4");
+
+    // Verify the default database is restricted when filtering Hive Privilege Objects
+    restrictDefaultDatabase(true);
+    filter = new MetastoreAuthzObjectFilter(mockBinding, HIVE_OBJECT_EXTRACTOR);
+    assertThat(filter.filterDatabases(USER1, createHivePrivilegeDatabaseList("db1", "default", "db4")))
+      .extracting(HiveObjectExtractor.objectName()).containsExactly("db4");
+
+    // Verify the default database is not restricted when filtering Hive Privilege Objects
+    restrictDefaultDatabase(false);
+    filter = new MetastoreAuthzObjectFilter(mockBinding, HIVE_OBJECT_EXTRACTOR);
+    assertThat(filter.filterDatabases(USER1, createHivePrivilegeDatabaseList("db1", "default", "db4")))
+      .extracting(HiveObjectExtractor.objectName()).containsExactly("default", "db4");
+  }
+
+  @Test
+  public void testFilterTableStrings() {
+    final String USER1 = "user1";
+    final String DB1 = "db1";
+
+    MetastoreAuthzObjectFilter filter =
+      new MetastoreAuthzObjectFilter(mockBinding, createTableStringExtractor(DB1));
+    assertThat(filter).isNotNull();
+
+    // Verify that null or empty lists do return an empty list (not null values to avoid NPE)
+    assertThat(filter.filterTables(USER1, null)).isNotNull().isEmpty();
+    assertThat(filter.filterTables(USER1, null)).isNotNull().isEmpty();
+    assertThat(filter.filterTables(USER1, Collections.emptyList())).isNotNull().isEmpty();
+
+    // Verify that null or empty items in the list are not filtered out and not cause exceptions
+    assertThat(filter.filterTables(USER1, Arrays.asList(null, "", null, null, "")))
+      .containsExactly(null, "", null, null, "");
+
+    // Verify restricted databases t1,t3,t5 are filtered out
+    restrictTablesNamesOnBinding(USER1, DB1, Arrays.asList("t1", "t3", "t5"));
+    assertThat(filter.filterTables(USER1, Arrays.asList("t1", "t2", "t3", "t4", "t5")))
+      .containsExactly("t2", "t4");
+  }
+
+  @Test
+  public void testFilterTableHiveObjects() {
+    final String USER1 = "user1";
+    final String DB1 = "db1";
+
+    MetastoreAuthzObjectFilter<HivePrivilegeObject>
+      filter = new MetastoreAuthzObjectFilter(mockBinding, HIVE_OBJECT_EXTRACTOR);
+    assertThat(filter).isNotNull();
+
+    // Verify that null or empty lists do return an empty list (not null values to avoid NPE)
+    assertThat(filter.filterTables(USER1, null)).isNotNull().isEmpty();
+    assertThat(filter.filterTables(USER1, null)).isNotNull().isEmpty();
+    assertThat(filter.filterTables(USER1, Collections.emptyList())).isNotNull().isEmpty();
+
+    // Verify that null or empty items in the list are not filtered out and not cause exceptions
+    assertThat(filter.filterTables(USER1, Arrays.asList(null, null)))
+      .containsExactly(null, null);
+
+    // Verify restricted databases t1,t3,t5 are filtered out
+    restrictTablesNamesOnBinding(USER1, DB1, Arrays.asList("t1", "t3", "t5"));
+    assertThat(filter.filterTables(USER1, createHivePrivilegeTableList(DB1, "t1", "t2", "t3", "t4", "t5")))
+      .extracting(HiveObjectExtractor.objectName()).containsExactly("t2", "t4");
+  }
+}

http://git-wip-us.apache.org/repos/asf/sentry/blob/3ba5998b/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/metastore/TestSentryMetaStoreFilterHook.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/metastore/TestSentryMetaStoreFilterHook.java b/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/metastore/TestSentryMetaStoreFilterHook.java
new file mode 100644
index 0000000..1f7148b
--- /dev/null
+++ b/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/metastore/TestSentryMetaStoreFilterHook.java
@@ -0,0 +1,219 @@
+/**
+ * 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 static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatExceptionOfType;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.List;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
+import org.apache.hadoop.hive.ql.plan.HiveOperation;
+import org.apache.sentry.binding.hive.authz.HiveAuthzBinding;
+import org.apache.sentry.binding.hive.authz.HiveAuthzPrivileges;
+import org.apache.sentry.binding.hive.authz.HiveAuthzPrivileges.HiveOperationScope;
+import org.apache.sentry.binding.hive.authz.HiveAuthzPrivileges.HiveOperationType;
+import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
+import org.apache.sentry.core.common.Subject;
+import org.apache.sentry.core.model.db.Column;
+import org.apache.sentry.core.model.db.DBModelAction;
+import org.apache.sentry.core.model.db.DBModelAuthorizable;
+import org.apache.sentry.core.model.db.DBModelAuthorizable.AuthorizableType;
+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.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+/**
+ * Testing classes for the {@code }SentryMetaStoreFilterHook}. {@code} SentryMetaStoreFilterHook}
+ * may be used by the HMS server to filter databases, tables and partitions that are authorized
+ * to be seen by a user making the HMS request.
+ */
+public class TestSentryMetaStoreFilterHook {
+  // Mock the HiveAuthzBinding to avoid making real connections to a Sentry server
+  private HiveAuthzBinding mockBinding = Mockito.mock(HiveAuthzBinding.class);
+  private final HiveAuthzPrivileges LIST_DATABASES_PRIVILEGES = new HiveAuthzPrivileges.AuthzPrivilegeBuilder()
+    .addInputObjectPriviledge(
+      AuthorizableType.Column,
+      EnumSet.of(
+        DBModelAction.SELECT, DBModelAction.INSERT, DBModelAction.ALTER,
+        DBModelAction.CREATE, DBModelAction.DROP, DBModelAction.INDEX,
+        DBModelAction.LOCK))
+    .addInputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.SELECT))
+    .setOperationScope(HiveOperationScope.CONNECT)
+    .setOperationType(HiveOperationType.QUERY)
+    .build();
+
+  private final HiveAuthzPrivileges LIST_TABLES_PRIVILEGES = new HiveAuthzPrivileges.AuthzPrivilegeBuilder()
+    .addInputObjectPriviledge(
+      AuthorizableType.Column,
+      EnumSet.of(
+        DBModelAction.SELECT, DBModelAction.INSERT, DBModelAction.ALTER,
+        DBModelAction.DROP, DBModelAction.INDEX, DBModelAction.LOCK))
+    .setOperationScope(HiveOperationScope.TABLE)
+    .setOperationType(
+      HiveAuthzPrivileges.HiveOperationType.INFO)
+    .build();
+
+  private HiveAuthzConf authzConf = new HiveAuthzConf();
+  private final Server SERVER1 = new Server("server1");
+
+  // Mock the authorize() method to throw an exception for the following list of databases
+  private void restrictDatabaseNamesOnBinding(String username, List<String> dbNames) {
+    for (String dbName : dbNames) {
+      Database database = new Database(dbName);
+      List<DBModelAuthorizable> authorizable = Arrays.asList(
+        SERVER1, database, Table.ALL, Column.ALL
+      );
+
+      Mockito.doThrow(new AuthorizationException())
+        .when(mockBinding).authorize(HiveOperation.SHOWDATABASES, LIST_DATABASES_PRIVILEGES, new Subject(username),
+        Collections.singleton(authorizable), Collections.emptySet());
+    }
+  }
+
+  // Mock the authorize() method to throw an exception for the following list of databases
+  private void restrictTablesNamesOnBinding(String username, String dbName, List<String> tableNames) {
+    Database database = new Database(dbName);
+
+    for (String tableName : tableNames) {
+      Table table = new Table(tableName);
+      List<DBModelAuthorizable> authorizable = Arrays.asList(
+        SERVER1, database, table, Column.ALL
+      );
+
+      Mockito.doThrow(new AuthorizationException())
+        .when(mockBinding).authorize(HiveOperation.SHOWTABLES,
+        LIST_TABLES_PRIVILEGES, new Subject(username), Collections.singleton(authorizable), Collections.emptySet());
+    }
+  }
+
+  // Returns a mock of the HiveAuthzBindingFactory with the userName wrapped inside.
+  private HiveAuthzBindingFactory getMockBinding(String userName) {
+    return new HiveAuthzBindingFactory() {
+      @Override
+      public HiveAuthzBinding fromMetaStoreConf(HiveConf hiveConf, HiveAuthzConf authzConf) throws Exception {
+        return mockBinding;
+      }
+
+      @Override
+      public String getUserName() {
+        return userName;
+      }
+    };
+  }
+
+  private org.apache.hadoop.hive.metastore.api.Database newHmsDatabase(String dbName) {
+    org.apache.hadoop.hive.metastore.api.Database db =
+      new org.apache.hadoop.hive.metastore.api.Database();
+
+    db.setName(dbName);
+    return db;
+  }
+
+  private org.apache.hadoop.hive.metastore.api.Table newHmsTable(String dbName, String tableName) {
+    org.apache.hadoop.hive.metastore.api.Table table =
+      new org.apache.hadoop.hive.metastore.api.Table();
+
+    table.setDbName(dbName);
+    table.setTableName(tableName);
+    return table;
+  }
+
+  @Before
+  public void setup() {
+    // Reset the mocks in case it was modified on the test methods
+    Mockito.reset(mockBinding);
+
+    Mockito.when(mockBinding.getAuthServer()).thenReturn(SERVER1);
+    Mockito.when(mockBinding.getAuthzConf()).thenReturn(authzConf);
+  }
+
+  @Test
+  public void testFilterListOfDatabases() {
+    final String USER1 = "user1";
+    SentryMetaStoreFilterHook filterHook = new SentryMetaStoreFilterHook(null, authzConf,
+      getMockBinding(USER1));
+
+    // Verify that only db2 is returned by the filter
+    restrictDatabaseNamesOnBinding(USER1, Arrays.asList("db1", "db3"));
+    assertThat(filterHook.filterDatabases(Arrays.asList("db1", "db2", "db3"))).containsExactly("db2");
+  }
+
+  @Test
+  public void testFilterSingleDatabase() throws NoSuchObjectException {
+    final String USER1 = "user1";
+    SentryMetaStoreFilterHook filterHook = new SentryMetaStoreFilterHook(null, authzConf,
+      getMockBinding(USER1));
+
+    restrictDatabaseNamesOnBinding(USER1, Arrays.asList("db1", "db3"));
+
+    // db1 and db3 must be denied
+    assertThatExceptionOfType(NoSuchObjectException.class)
+      .isThrownBy(() -> filterHook.filterDatabase(newHmsDatabase("db1")));
+    assertThatExceptionOfType(NoSuchObjectException.class)
+      .isThrownBy(() -> filterHook.filterDatabase(newHmsDatabase("db3")));
+
+    // db2 must be allowed
+    assertThat(filterHook.filterDatabase(newHmsDatabase("db2")))
+      .isEqualTo(newHmsDatabase("db2"));
+  }
+
+  @Test
+  public void testFilterListOfTables() {
+    final String USER1 = "user1";
+    final String DB1 = "db1";
+    SentryMetaStoreFilterHook filterHook = new SentryMetaStoreFilterHook(null, authzConf,
+      getMockBinding(USER1));
+
+    // Verify that only db2 is returned by the filter
+    restrictTablesNamesOnBinding(USER1, DB1, Arrays.asList("t1", "t3"));
+    assertThat(filterHook.filterTableNames(DB1, Arrays.asList("t1", "t2", "t3"))).containsExactly("t2");
+    assertThat(filterHook.filterTables(Arrays.asList(
+      newHmsTable(DB1, "t1"),
+      newHmsTable(DB1, "t2"),
+      newHmsTable(DB1, "t3")
+    ))).containsExactly(newHmsTable(DB1, "t2"));
+  }
+
+  @Test
+  public void testFilterSingleTable() throws NoSuchObjectException {
+    final String USER1 = "user1";
+    final String DB1 = "db1";
+    SentryMetaStoreFilterHook filterHook = new SentryMetaStoreFilterHook(null, authzConf,
+      getMockBinding(USER1));
+
+    restrictTablesNamesOnBinding(USER1, DB1, Arrays.asList("t1", "t3"));
+
+    // t1 and t3 must be denied
+    assertThatExceptionOfType(NoSuchObjectException.class)
+      .isThrownBy(() -> filterHook.filterTable(newHmsTable(DB1, "t1")));
+    assertThatExceptionOfType(NoSuchObjectException.class)
+      .isThrownBy(() -> filterHook.filterTable(newHmsTable(DB1, "t3")));
+
+    // t2 must be allowed
+    assertThat(filterHook.filterTable(newHmsTable(DB1, "t2")))
+      .isEqualTo(newHmsTable(DB1, "t2"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/sentry/blob/3ba5998b/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/Subject.java
----------------------------------------------------------------------
diff --git a/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/Subject.java b/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/Subject.java
index bcd1fa2..f93fb9d 100644
--- a/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/Subject.java
+++ b/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/Subject.java
@@ -16,6 +16,7 @@
  */
 package org.apache.sentry.core.common;
 
+import java.util.Objects;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 
 @Public
@@ -27,6 +28,23 @@ public class Subject {
     this.name = name;
   }
 
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof Subject)) {
+      return false;
+    }
+    Subject subject = (Subject) o;
+    return Objects.equals(name, subject.name);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(name);
+  }
+
   public String getName() {
     return name;
   }