You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ng...@apache.org on 2020/08/04 13:12:11 UTC

[hive] branch master updated: HIVE-23786 HMS server side filter with Ranger (Sam An reviewed by Peter Vary)

This is an automated email from the ASF dual-hosted git repository.

ngangam pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new 1a1fd15  HIVE-23786 HMS server side filter with Ranger (Sam An reviewed by Peter Vary)
1a1fd15 is described below

commit 1a1fd15e3daa69e6dd5816ec877c3e8634cb7718
Author: Sam An <sa...@cloudera.com>
AuthorDate: Tue Jul 7 10:34:08 2020 -0700

    HIVE-23786 HMS server side filter with Ranger (Sam An reviewed by Peter Vary)
---
 .../plugin/metastore/HiveMetaStoreAuthorizer.java  | 367 ++++++++++++++++++---
 .../plugin/metastore/HiveMetaStoreAuthzInfo.java   |  88 ++---
 .../plugin/metastore/events/ReadDatabaseEvent.java |  80 +++++
 .../plugin/metastore/events/ReadTableEvent.java    |  73 ++++
 .../filtercontext/DatabaseFilterContext.java       |  76 +++++
 .../filtercontext/TableFilterContext.java          |  98 ++++++
 .../metastore/TestHiveMetaStoreAuthorizer.java     |  47 +++
 7 files changed, 735 insertions(+), 94 deletions(-)

diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/HiveMetaStoreAuthorizer.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/HiveMetaStoreAuthorizer.java
index 434d1c9..09d5946 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/HiveMetaStoreAuthorizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/HiveMetaStoreAuthorizer.java
@@ -18,10 +18,12 @@
  */
 package org.apache.hadoop.hive.ql.security.authorization.plugin.metastore;
 
+import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.MetaStoreFilterHook;
 import org.apache.hadoop.hive.metastore.HiveMetaStore;
 import org.apache.hadoop.hive.metastore.MetaStorePreEventListener;
 import org.apache.hadoop.hive.metastore.TableType;
@@ -34,6 +36,11 @@ import org.apache.hadoop.hive.metastore.events.PreCreateTableEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropTableEvent;
 import org.apache.hadoop.hive.metastore.events.PreEventContext;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.PartitionSpec;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.HiveUtils;
 import org.apache.hadoop.hive.ql.security.HiveMetastoreAuthenticationProvider;
@@ -45,17 +52,21 @@ import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzSessionC
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveMetastoreClientFactoryImpl;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.metastore.filtercontext.DatabaseFilterContext;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.metastore.filtercontext.TableFilterContext;
 import org.apache.hadoop.security.UserGroupInformation;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 
 /**
  * HiveMetaStoreAuthorizer :  Do authorization checks on MetaStore Events in MetaStorePreEventListener
  */
 
-public class HiveMetaStoreAuthorizer extends MetaStorePreEventListener {
-  private static final Log    LOG              = LogFactory.getLog(HiveMetaStoreAuthorizer.class);
+public class HiveMetaStoreAuthorizer extends MetaStorePreEventListener implements MetaStoreFilterHook {
+  private static final Log LOG = LogFactory.getLog(HiveMetaStoreAuthorizer.class);
 
   private static final ThreadLocal<Configuration> tConfig = new ThreadLocal<Configuration>() {
     @Override
@@ -80,50 +91,278 @@ public class HiveMetaStoreAuthorizer extends MetaStorePreEventListener {
   }
 
   @Override
-  public final void onEvent(PreEventContext preEventContext) throws MetaException, NoSuchObjectException, InvalidOperationException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("==> HiveMetaStoreAuthorizer.onEvent(): EventType=" + preEventContext.getEventType());
+  public final void onEvent(PreEventContext preEventContext)
+      throws MetaException, NoSuchObjectException, InvalidOperationException {
+    LOG.debug("==> HiveMetaStoreAuthorizer.onEvent(): EventType=" + preEventContext.getEventType());
+
+    try {
+      HiveAuthorizer hiveAuthorizer = createHiveMetaStoreAuthorizer();
+      if (!skipAuthorization()) {
+        HiveMetaStoreAuthzInfo authzContext = buildAuthzContext(preEventContext);
+        checkPrivileges(authzContext, hiveAuthorizer);
+      }
+    } catch (Exception e) {
+      LOG.error("HiveMetaStoreAuthorizer.onEvent(): failed", e);
+      throw new MetaException(e.getMessage());
     }
 
-    HiveMetaStoreAuthzInfo authzContext = buildAuthzContext(preEventContext);
+    LOG.debug("<== HiveMetaStoreAuthorizer.onEvent(): EventType=" + preEventContext.getEventType());
+  }
 
-    if (!skipAuthorization(authzContext)) {
-      try {
-        HiveConf              hiveConf          = new HiveConf(super.getConf(), HiveConf.class);
-        HiveAuthorizerFactory authorizerFactory = HiveUtils.getAuthorizerFactory(hiveConf, HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER);
+  @Override
+  public final List<String> filterDatabases(List<String> list) throws MetaException {
+    LOG.debug("HiveMetaStoreAuthorizer.filterDatabases()");
 
-        if (authorizerFactory != null) {
-          HiveMetastoreAuthenticationProvider authenticator = tAuthenticator.get();
+    if (list == null) {
+      return Collections.emptyList();
+    }
 
-          authenticator.setConf(hiveConf);
+    DatabaseFilterContext databaseFilterContext = new DatabaseFilterContext(list);
+    HiveMetaStoreAuthzInfo hiveMetaStoreAuthzInfo = databaseFilterContext.getAuthzContext();
+    List<String> filteredDatabases = filterDatabaseObjects(hiveMetaStoreAuthzInfo);
+    if (CollectionUtils.isEmpty(filteredDatabases)) {
+      filteredDatabases = Collections.emptyList();
+    }
 
-          HiveAuthzSessionContext.Builder authzContextBuilder = new HiveAuthzSessionContext.Builder();
+    LOG.debug("HiveMetaStoreAuthorizer.filterDatabases() :" + filteredDatabases);
 
-          authzContextBuilder.setClientType(HiveAuthzSessionContext.CLIENT_TYPE.HIVEMETASTORE);
-          authzContextBuilder.setSessionString("HiveMetaStore");
+    return filteredDatabases;
+  }
 
-          HiveAuthzSessionContext authzSessionContext = authzContextBuilder.build();
+  @Override
+  public final Database filterDatabase(Database database) throws MetaException, NoSuchObjectException {
+    if (database != null) {
+      String dbName = database.getName();
+      List<String> databases = filterDatabases(Collections.singletonList(dbName));
+      if (databases.isEmpty()) {
+        throw new NoSuchObjectException(String.format("Database %s does not exist", dbName));
+      }
+    }
+    return database;
+  }
 
-          HiveAuthorizer hiveAuthorizer = authorizerFactory.createHiveAuthorizer(new HiveMetastoreClientFactoryImpl(), hiveConf, authenticator, authzSessionContext);
+  @Override
+  public final List<String> filterTableNames(String s, String s1, List<String> list) throws MetaException {
+    LOG.debug("==> HiveMetaStoreAuthorizer.filterTableNames()");
+
+    List<String> filteredTableNames = null;
+    if (list != null) {
+      String dbName = getDBName(s1);
+      TableFilterContext tableFilterContext = new TableFilterContext(dbName, list);
+      HiveMetaStoreAuthzInfo hiveMetaStoreAuthzInfo = tableFilterContext.getAuthzContext();
+      filteredTableNames = filterTableNames(hiveMetaStoreAuthzInfo, dbName, list);
+      if (CollectionUtils.isEmpty(filteredTableNames)) {
+        filteredTableNames = Collections.emptyList();
+      }
+    }
+
+    LOG.debug("<== HiveMetaStoreAuthorizer.filterTableNames() : " + filteredTableNames);
+
+    return filteredTableNames;
+  }
 
-          checkPrivileges(authzContext, hiveAuthorizer);
-        }
-      } catch (Exception e) {
-        LOG.error("HiveMetaStoreAuthorizer.onEvent(): failed", e);
-        throw new MetaException(e.getMessage());
+  @Override
+  public final Table filterTable(Table table) throws MetaException, NoSuchObjectException {
+    if (table != null) {
+      List<Table> tables = filterTables(Collections.singletonList(table));
+      if (tables.isEmpty()) {
+        throw new NoSuchObjectException(String.format("Database %s does not exist", table.getTableName()));
       }
     }
+    return table;
+  }
+
+  @Override
+  public final List<Table> filterTables(List<Table> list) throws MetaException {
+    LOG.debug("==> HiveMetaStoreAuthorizer.filterTables()");
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("<== HiveMetaStoreAuthorizer.onEvent(): EventType=" + preEventContext.getEventType());
+    List<Table> filteredTables = null;
+
+    if (list != null) {
+      TableFilterContext tableFilterContext = new TableFilterContext(list);
+      HiveMetaStoreAuthzInfo hiveMetaStoreAuthzInfo = tableFilterContext.getAuthzContext();
+      filteredTables = filterTableObjects(hiveMetaStoreAuthzInfo, list);
+      if (CollectionUtils.isEmpty(filteredTables)) {
+        filteredTables = Collections.emptyList();
+      }
     }
+
+    LOG.debug("<== HiveMetaStoreAuthorizer.filterTables(): " + filteredTables);
+
+    return filteredTables;
   }
 
-  HiveMetaStoreAuthzInfo buildAuthzContext(PreEventContext preEventContext) throws MetaException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("==> HiveMetaStoreAuthorizer.buildAuthzContext(): EventType=" + preEventContext.getEventType());
+  @Override
+  public final Catalog filterCatalog(Catalog catalog) throws MetaException {
+    return catalog;
+  }
+
+  @Override
+  public final List<String> filterCatalogs(List<String> catalogs) throws MetaException {
+    return catalogs;
+  }
+
+  @Override
+  public final List<TableMeta> filterTableMetas(String catName, String dbName, List<TableMeta> tableMetas)
+      throws MetaException {
+    return tableMetas;
+  }
+
+  @Override
+  public final List<Partition> filterPartitions(List<Partition> list) throws MetaException {
+    return list;
+  }
+
+  @Override
+  public final List<PartitionSpec> filterPartitionSpecs(List<PartitionSpec> list) throws MetaException {
+    return list;
+  }
+
+  @Override
+  public final Partition filterPartition(Partition partition) throws MetaException, NoSuchObjectException {
+    return partition;
+  }
+
+  @Override
+  public final List<String> filterPartitionNames(String s, String s1, String s2, List<String> list)
+      throws MetaException {
+    return list;
+  }
+
+  private List<String> filterDatabaseObjects(HiveMetaStoreAuthzInfo hiveMetaStoreAuthzInfo) throws MetaException {
+    List<String> ret = null;
+
+    LOG.debug("==> HiveMetaStoreAuthorizer.filterDatabaseObjects()");
+
+    try {
+      HiveAuthorizer hiveAuthorizer = createHiveMetaStoreAuthorizer();
+      List<HivePrivilegeObject> hivePrivilegeObjects = hiveMetaStoreAuthzInfo.getInputHObjs();
+      HiveAuthzContext hiveAuthzContext = hiveMetaStoreAuthzInfo.getHiveAuthzContext();
+      List<HivePrivilegeObject> filteredHivePrivilegeObjects =
+          hiveAuthorizer.filterListCmdObjects(hivePrivilegeObjects, hiveAuthzContext);
+      if (CollectionUtils.isNotEmpty(filteredHivePrivilegeObjects)) {
+        ret = getFilteredDatabaseList(filteredHivePrivilegeObjects);
+      }
+      LOG.info(String.format("Filtered %d databases out of %d", filteredHivePrivilegeObjects.size(),
+          hivePrivilegeObjects.size()));
+    } catch (Exception e) {
+      throw new MetaException("Error in HiveMetaStoreAuthorizer.filterDatabase()" + e.getMessage());
     }
 
+    LOG.debug("<== HiveMetaStoreAuthorizer.filterDatabaseObjects() :" + ret );
+
+    return ret;
+  }
+
+  private List<Table> filterTableObjects(HiveMetaStoreAuthzInfo hiveMetaStoreAuthzInfo, List<Table> tableList)
+      throws MetaException {
+    List<Table> ret = null;
+
+    try {
+      HiveAuthorizer hiveAuthorizer = createHiveMetaStoreAuthorizer();
+      List<HivePrivilegeObject> hivePrivilegeObjects = hiveMetaStoreAuthzInfo.getInputHObjs();
+      HiveAuthzContext hiveAuthzContext = hiveMetaStoreAuthzInfo.getHiveAuthzContext();
+      List<HivePrivilegeObject> filteredHivePrivilegeObjects =
+          hiveAuthorizer.filterListCmdObjects(hivePrivilegeObjects, hiveAuthzContext);
+      if (CollectionUtils.isNotEmpty(filteredHivePrivilegeObjects)) {
+        ret = getFilteredTableList(filteredHivePrivilegeObjects, tableList);
+      }
+      LOG.info(String.format("Filtered %d tables out of %d", filteredHivePrivilegeObjects.size(),
+          hivePrivilegeObjects.size()));
+    } catch (Exception e) {
+      throw new MetaException("Error in HiveMetaStoreAuthorizer.filterTables()" + e.getMessage());
+    }
+    return ret;
+  }
+
+  private List<String> getFilteredDatabaseList(List<HivePrivilegeObject> hivePrivilegeObjects) {
+    List<String> ret = new ArrayList<>();
+    for(HivePrivilegeObject hivePrivilegeObject: hivePrivilegeObjects) {
+      String dbName = hivePrivilegeObject.getDbname();
+      ret.add(dbName);
+    }
+    return ret;
+  }
+
+  private List<Table> getFilteredTableList(List<HivePrivilegeObject> hivePrivilegeObjects, List<Table> tableList) {
+    List<Table> ret = new ArrayList<>();
+    for (HivePrivilegeObject hivePrivilegeObject : hivePrivilegeObjects) {
+      String dbName = hivePrivilegeObject.getDbname();
+      String tblName = hivePrivilegeObject.getObjectName();
+      Table table = getFilteredTable(dbName, tblName, tableList);
+      if (table != null) {
+        ret.add(table);
+      }
+    }
+    return ret;
+  }
+
+  private Table getFilteredTable(String dbName, String tblName, List<Table> tableList) {
+    Table ret = null;
+    for (Table table: tableList) {
+      String databaseName = table.getDbName();
+      String tableName = table.getTableName();
+      if (dbName.equals(databaseName) && tblName.equals(tableName)) {
+        ret = table;
+        break;
+      }
+    }
+    return ret;
+  }
+
+  private List<String> filterTableNames(HiveMetaStoreAuthzInfo hiveMetaStoreAuthzInfo, String dbName,
+      List<String> tableNames) throws MetaException {
+    List<String> ret = null;
+
+    try {
+      HiveAuthorizer hiveAuthorizer = createHiveMetaStoreAuthorizer();
+      List<HivePrivilegeObject> hivePrivilegeObjects = hiveMetaStoreAuthzInfo.getInputHObjs();
+      HiveAuthzContext hiveAuthzContext = hiveMetaStoreAuthzInfo.getHiveAuthzContext();
+      List<HivePrivilegeObject> filteredHivePrivilegeObjects =
+          hiveAuthorizer.filterListCmdObjects(hivePrivilegeObjects, hiveAuthzContext);
+      if (CollectionUtils.isNotEmpty(filteredHivePrivilegeObjects)) {
+        ret = getFilteredTableNames(filteredHivePrivilegeObjects, dbName, tableNames);
+      }
+      LOG.info(String.format("Filtered %d table names out of %d", filteredHivePrivilegeObjects.size(),
+          hivePrivilegeObjects.size()));
+    } catch (Exception e) {
+      throw new MetaException("Error in HiveMetaStoreAuthorizer.filterTables()" + e.getMessage());
+    }
+    return ret;
+  }
+
+  private List<String> getFilteredTableNames(List<HivePrivilegeObject> hivePrivilegeObjects, String databaseName,
+      List<String> tableNames) {
+    List<String> ret = new ArrayList<>();
+    for (HivePrivilegeObject hivePrivilegeObject : hivePrivilegeObjects) {
+      String dbName = hivePrivilegeObject.getDbname();
+      String tblName = hivePrivilegeObject.getObjectName();
+      String table = getFilteredTableNames(dbName, tblName, databaseName, tableNames);
+      if (table != null) {
+        ret.add(table);
+      }
+    }
+    return ret;
+  }
+
+  private String getFilteredTableNames(String dbName, String tblName, String databaseName, List<String> tableNames) {
+    String ret = null;
+    for (String tableName : tableNames) {
+      if (dbName.equals(databaseName) && tblName.equals(tableName)) {
+        ret = tableName;
+        break;
+      }
+    }
+    return ret;
+  }
+
+  private String getDBName(String str) {
+   return (str != null) ? str.substring(str.indexOf("#")+1) : null;
+  }
+
+  HiveMetaStoreAuthzInfo buildAuthzContext(PreEventContext preEventContext) throws MetaException {
+    LOG.debug("==> HiveMetaStoreAuthorizer.buildAuthzContext(): EventType=" + preEventContext.getEventType());
+
     HiveMetaStoreAuthorizableEvent authzEvent = null;
 
     if (preEventContext != null) {
@@ -168,6 +407,12 @@ public class HiveMetaStoreAuthorizer extends MetaStorePreEventListener {
         case DROP_PARTITION:
           authzEvent = new DropPartitionEvent(preEventContext);
           break;
+        case READ_TABLE:
+          authzEvent = new ReadTableEvent(preEventContext);
+          break;
+        case READ_DATABASE:
+          authzEvent = new ReadDatabaseEvent(preEventContext);
+          break;
         case AUTHORIZATION_API_CALL:
         case READ_ISCHEMA:
         case CREATE_ISCHEMA:
@@ -191,8 +436,31 @@ public class HiveMetaStoreAuthorizer extends MetaStorePreEventListener {
 
     HiveMetaStoreAuthzInfo ret = authzEvent != null ? authzEvent.getAuthzContext() : null;
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("<== HiveMetaStoreAuthorizer.buildAuthzContext(): EventType=" + preEventContext.getEventType() + "; ret=" + ret);
+    LOG.debug("<== HiveMetaStoreAuthorizer.buildAuthzContext(): EventType=" + preEventContext.getEventType() + "; ret=" + ret);
+
+    return ret;
+  }
+
+  HiveAuthorizer createHiveMetaStoreAuthorizer() throws Exception {
+    HiveAuthorizer ret = null;
+    HiveConf hiveConf = new HiveConf(super.getConf(), HiveConf.class);
+    HiveAuthorizerFactory authorizerFactory =
+        HiveUtils.getAuthorizerFactory(hiveConf, HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER);
+
+    if (authorizerFactory != null) {
+      HiveMetastoreAuthenticationProvider authenticator = tAuthenticator.get();
+
+      authenticator.setConf(hiveConf);
+
+      HiveAuthzSessionContext.Builder authzContextBuilder = new HiveAuthzSessionContext.Builder();
+
+      authzContextBuilder.setClientType(HiveAuthzSessionContext.CLIENT_TYPE.HIVEMETASTORE);
+      authzContextBuilder.setSessionString("HiveMetaStore");
+
+      HiveAuthzSessionContext authzSessionContext = authzContextBuilder.build();
+
+      ret = authorizerFactory
+          .createHiveAuthorizer(new HiveMetastoreClientFactoryImpl(), hiveConf, authenticator, authzSessionContext);
     }
 
     return ret;
@@ -232,9 +500,7 @@ public class HiveMetaStoreAuthorizer extends MetaStorePreEventListener {
   }
 
   private void checkPrivileges(final HiveMetaStoreAuthzInfo authzContext, HiveAuthorizer authorizer) throws MetaException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("==> HiveMetaStoreAuthorizer.checkPrivileges(): authzContext=" + authzContext + ", authorizer=" + authorizer);
-    }
+    LOG.debug("==> HiveMetaStoreAuthorizer.checkPrivileges(): authzContext=" + authzContext + ", authorizer=" + authorizer);
 
     HiveOperationType         hiveOpType       = authzContext.getOperationType();
     List<HivePrivilegeObject> inputHObjs       = authzContext.getInputHObjs();
@@ -247,34 +513,23 @@ public class HiveMetaStoreAuthorizer extends MetaStorePreEventListener {
       throw new MetaException(e.getMessage());
     }
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("<== HiveMetaStoreAuthorizer.checkPrivileges(): authzContext=" + authzContext + ", authorizer=" + authorizer);
-    }
+    LOG.debug("<== HiveMetaStoreAuthorizer.checkPrivileges(): authzContext=" + authzContext + ", authorizer=" + authorizer);
   }
 
-  private boolean skipAuthorization(HiveMetaStoreAuthzInfo authzContext) {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("==> HiveMetaStoreAuthorizer.skipAuthorization(): authzContext=" + authzContext);
-    }
+  private boolean skipAuthorization() {
+    LOG.debug("==> HiveMetaStoreAuthorizer.skipAuthorization()");
 
     boolean ret = false;
 
-    if (authzContext == null) {
-      ret = true;
-    } else {
-
-      UserGroupInformation ugi = authzContext.getUGI();
-
-      if (ugi == null) {
-        ret = true;
-      } else {
-        ret = isSuperUser(ugi.getShortUserName());
-      }
+    UserGroupInformation ugi = null;
+    try {
+      ugi = getUGI();
+      ret = isSuperUser(ugi.getShortUserName());
+    } catch (IOException e) {
+      LOG.warn("Not able to obtain UserGroupInformation", e);
     }
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("<== HiveMetaStoreAuthorizer.skipAuthorization(): authzContext=" + authzContext + "; ret=" + ret);
-    }
+    LOG.debug("<== HiveMetaStoreAuthorizer.skipAuthorization(): " + ret);
 
     return ret;
   }
@@ -312,5 +567,9 @@ public class HiveMetaStoreAuthorizer extends MetaStorePreEventListener {
   private String getCurrentUser(HiveMetaStoreAuthorizableEvent authorizableEvent) {
     return authorizableEvent.getAuthzContext().getUGI().getShortUserName();
   }
+
+  private UserGroupInformation getUGI() throws IOException {
+    return UserGroupInformation.getCurrentUser();
+  }
 }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/HiveMetaStoreAuthzInfo.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/HiveMetaStoreAuthzInfo.java
index a372c78..fadccb3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/HiveMetaStoreAuthzInfo.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/HiveMetaStoreAuthzInfo.java
@@ -19,6 +19,7 @@
 
 package org.apache.hadoop.hive.ql.security.authorization.plugin.metastore;
 
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hive.metastore.HiveMetaStore;
 import org.apache.hadoop.hive.metastore.events.PreEventContext;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext;
@@ -50,58 +51,65 @@ public class HiveMetaStoreAuthzInfo {
     this.hiveAuthzContext = createHiveAuthzContext();
   }
 
-    public HiveOperationType getOperationType() {
-        return operationType;
-    }
+  public HiveOperationType getOperationType() {
+    return operationType;
+  }
 
-    public List<HivePrivilegeObject> getInputHObjs() { return inputHObjs; }
+  public List<HivePrivilegeObject> getInputHObjs() {
+    return inputHObjs;
+  }
 
-    public List<HivePrivilegeObject> getOutputHObjs() { return outputHObjs; }
+  public List<HivePrivilegeObject> getOutputHObjs() {
+    return outputHObjs;
+  }
 
-    public String getCommandString() {
-        return commandString;
-    }
+  public String getCommandString() {
+    return commandString;
+  }
 
-    public HiveAuthzContext getHiveAuthzContext() { return hiveAuthzContext; }
+  public HiveAuthzContext getHiveAuthzContext() {
+    return hiveAuthzContext;
+  }
 
-    public PreEventContext getPreEventContext(){
-      return preEventContext;
-    }
+  public PreEventContext getPreEventContext() {
+    return preEventContext;
+  }
 
-    public UserGroupInformation getUGI() {
-      try {
-        return UserGroupInformation.getCurrentUser();
-        } catch (IOException excp) {
-      }
-      return null;
+  public UserGroupInformation getUGI() {
+    try {
+      return UserGroupInformation.getCurrentUser();
+    } catch (IOException excp) {
     }
+    return null;
+  }
 
-    @Override
-    public String toString() {
-      StringBuilder sb = new StringBuilder();
-      sb.append("HiveMetaStoreAuthzInfo= ").append("{");
-      sb.append("eventType=").append(preEventContext.getEventType().name());
-      sb.append(", operationType=").append(operationType.name());
-      sb.append(", commandString=" ).append(commandString);
-      sb.append(", inputHObjs=").append(inputHObjs);
-      sb.append(", outputHObjs=").append(outputHObjs);
-      sb.append(" }");
-      return sb.toString();
-    }
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("HiveMetaStoreAuthzInfo= ").append("{");
+    String eventType = (preEventContext != null) ? preEventContext.getEventType().name() : StringUtils.EMPTY;
+    sb.append("eventType=").append(eventType);
+    sb.append(", operationType=").append(operationType.name());
+    sb.append(", commandString=").append(commandString);
+    sb.append(", inputHObjs=").append(inputHObjs);
+    sb.append(", outputHObjs=").append(outputHObjs);
+    sb.append(" }");
+    return sb.toString();
+  }
 
-    private HiveAuthzContext createHiveAuthzContext() {
-      HiveAuthzContext.Builder builder = new HiveAuthzContext.Builder();
-      builder.setCommandString(commandString);
+  private HiveAuthzContext createHiveAuthzContext() {
+    HiveAuthzContext.Builder builder = new HiveAuthzContext.Builder();
+    builder.setCommandString(commandString);
 
-      // TODO: refer to SessionManager/HiveSessionImpl for details on getting ipAddress and forwardedAddresses
-      builder.setForwardedAddresses(new ArrayList<>());
+    // TODO: refer to SessionManager/HiveSessionImpl for details on getting ipAddress and forwardedAddresses
+    builder.setForwardedAddresses(new ArrayList<>());
 
-      String ipAddress = HiveMetaStore.HMSHandler.getIPAddress();
+    String ipAddress = HiveMetaStore.HMSHandler.getIPAddress();
 
-      builder.setUserIpAddress(ipAddress);
+    builder.setUserIpAddress(ipAddress);
 
-      HiveAuthzContext ret = builder.build();
+    HiveAuthzContext ret = builder.build();
 
-      return ret;
-    }
+    return ret;
+  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/events/ReadDatabaseEvent.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/events/ReadDatabaseEvent.java
new file mode 100644
index 0000000..0af71f5
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/events/ReadDatabaseEvent.java
@@ -0,0 +1,80 @@
+/*
+ * 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.security.authorization.plugin.metastore.events;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.events.PreEventContext;
+import org.apache.hadoop.hive.metastore.events.PreReadDatabaseEvent;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.metastore.HiveMetaStoreAuthorizableEvent;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.metastore.HiveMetaStoreAuthzInfo;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class ReadDatabaseEvent extends HiveMetaStoreAuthorizableEvent {
+  private static final Log LOG = LogFactory.getLog(ReadDatabaseEvent.class);
+  private String COMMAND_STR = "use/show databases or tables";
+  public ReadDatabaseEvent(PreEventContext preEventContext) {
+    super(preEventContext);
+  }
+
+  @Override public HiveMetaStoreAuthzInfo getAuthzContext() {
+    HiveMetaStoreAuthzInfo ret =
+        new HiveMetaStoreAuthzInfo(preEventContext, HiveOperationType.QUERY, getInputHObjs(), getOutputHObjs(),
+            COMMAND_STR);
+    return ret;
+  }
+
+  private List<HivePrivilegeObject> getInputHObjs() {
+    LOG.debug("==> ReadDatabaseEvent.getInputHObjs()");
+
+    List<HivePrivilegeObject> ret = new ArrayList<>();
+    PreReadDatabaseEvent preReadDatabaseEvent = (PreReadDatabaseEvent) preEventContext;
+    Database database = preReadDatabaseEvent.getDatabase();
+    if (database != null) {
+      ret.add(getHivePrivilegeObject(database));
+
+      COMMAND_STR = buildCommandString(COMMAND_STR, database);
+
+      LOG.debug("<== ReadDatabaseEvent.getInputHObjs(): ret=" + ret);
+    }
+    return ret;
+  }
+
+  private List<HivePrivilegeObject> getOutputHObjs() {
+    return Collections.emptyList();
+  }
+
+  private String buildCommandString(String cmdStr, Database db) {
+    String ret = cmdStr;
+
+    if (db != null) {
+      String dbName = db.getName();
+      ret = ret + (StringUtils.isNotEmpty(dbName) ? " " + dbName : "");
+    }
+
+    return ret;
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/events/ReadTableEvent.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/events/ReadTableEvent.java
new file mode 100644
index 0000000..21152c3
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/events/ReadTableEvent.java
@@ -0,0 +1,73 @@
+/*
+ * 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.security.authorization.plugin.metastore.events;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.events.PreEventContext;
+import org.apache.hadoop.hive.metastore.events.PreReadTableEvent;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.metastore.HiveMetaStoreAuthorizableEvent;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.metastore.HiveMetaStoreAuthzInfo;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class ReadTableEvent extends HiveMetaStoreAuthorizableEvent {
+  private static final Log LOG = LogFactory.getLog(ReadTableEvent.class);
+  private static final String COMMAND_STR = "select";
+
+  public ReadTableEvent(PreEventContext preEventContext) {
+    super(preEventContext);
+  }
+
+  @Override public HiveMetaStoreAuthzInfo getAuthzContext() {
+    HiveMetaStoreAuthzInfo ret =
+        new HiveMetaStoreAuthzInfo(preEventContext, HiveOperationType.QUERY, getInputHObjs(), getOutputHObjs(),
+            COMMAND_STR);
+    return ret;
+  }
+
+  private List<HivePrivilegeObject> getInputHObjs() {
+    LOG.debug("==> ReadTableEvent.getInputHObjs()");
+
+    List<HivePrivilegeObject> ret = new ArrayList<>();
+    PreReadTableEvent preReadTableEvent = (PreReadTableEvent) preEventContext;
+    String dbName = preReadTableEvent.getTable().getDbName();
+    Table table = preReadTableEvent.getTable();
+
+    ret.add(new HivePrivilegeObject(HivePrivilegeObjectType.DATABASE, dbName, null, null, null,
+        HivePrivilegeObject.HivePrivObjectActionType.OTHER, null, null, table.getOwner(), table.getOwnerType()));
+    ret.add(getHivePrivilegeObject(table));
+
+    LOG.debug("<== ReadTableEvent.getInputHObjs()" + ret);
+
+    return ret;
+  }
+
+  private List<HivePrivilegeObject> getOutputHObjs() {
+    return Collections.emptyList();
+  }
+
+}
\ No newline at end of file
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/filtercontext/DatabaseFilterContext.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/filtercontext/DatabaseFilterContext.java
new file mode 100644
index 0000000..da2a3d2
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/filtercontext/DatabaseFilterContext.java
@@ -0,0 +1,76 @@
+/*
+ * 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.security.authorization.plugin.metastore.filtercontext;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivObjectActionType;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.metastore.HiveMetaStoreAuthorizableEvent;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.metastore.HiveMetaStoreAuthzInfo;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class DatabaseFilterContext extends HiveMetaStoreAuthorizableEvent {
+
+  private static final Log LOG = LogFactory.getLog(DatabaseFilterContext.class);
+
+  List<String> databases = null;
+
+  public DatabaseFilterContext(List<String> databases) {
+    super(null);
+    this.databases = databases;
+    getAuthzContext();
+  }
+
+  @Override public HiveMetaStoreAuthzInfo getAuthzContext() {
+    HiveMetaStoreAuthzInfo ret =
+        new HiveMetaStoreAuthzInfo(preEventContext, HiveOperationType.QUERY, getInputHObjs(), getOutputHObjs(), null);
+    return ret;
+  }
+
+  private List<HivePrivilegeObject> getInputHObjs() {
+    LOG.debug("==> DatabaseFilterContext.getOutputHObjs()");
+
+    List<HivePrivilegeObject> ret = new ArrayList<>();
+    for (String database : databases) {
+      HivePrivilegeObjectType type = HivePrivilegeObjectType.DATABASE;
+      HivePrivObjectActionType objectActionType = HivePrivObjectActionType.OTHER;
+      HivePrivilegeObject hivePrivilegeObject =
+          new HivePrivilegeObject(type, database, null, null, null, objectActionType, null, null);
+      ret.add(hivePrivilegeObject);
+    }
+    LOG.debug("<== DatabaseFilterContext.getOutputHObjs(): ret=" + ret);
+
+    return ret;
+  }
+
+  private List<HivePrivilegeObject> getOutputHObjs() {
+    return Collections.emptyList();
+  }
+
+  public List<String> getDatabases() {
+    return databases;
+  }
+}
\ No newline at end of file
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/filtercontext/TableFilterContext.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/filtercontext/TableFilterContext.java
new file mode 100644
index 0000000..b140200
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/filtercontext/TableFilterContext.java
@@ -0,0 +1,98 @@
+
+/*
+ * 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.security.authorization.plugin.metastore.filtercontext;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivObjectActionType;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.metastore.HiveMetaStoreAuthorizableEvent;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.metastore.HiveMetaStoreAuthzInfo;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class TableFilterContext extends HiveMetaStoreAuthorizableEvent {
+  private static final Log LOG = LogFactory.getLog(DatabaseFilterContext.class);
+
+  List<Table> tables = null;
+  List<String> tableNames = null;
+  String dbName = null;
+
+  public TableFilterContext(List<Table> tables) {
+    super(null);
+    this.tables = tables;
+    getAuthzContext();
+  }
+
+  public TableFilterContext(String dbName, List<String> tableNames) {
+    super(null);
+    this.dbName = dbName;
+    this.tableNames = tableNames;
+  }
+
+  @Override public HiveMetaStoreAuthzInfo getAuthzContext() {
+    HiveMetaStoreAuthzInfo ret =
+        new HiveMetaStoreAuthzInfo(preEventContext, HiveOperationType.QUERY, getInputHObjs(), getOutputHObjs(), null);
+    return ret;
+  }
+
+  private List<HivePrivilegeObject> getInputHObjs() {
+    LOG.debug("==> TableFilterContext.getOutputHObjs()");
+
+    List<HivePrivilegeObject> ret = new ArrayList<>();
+
+    if (tables != null) {
+      for (Table table : tables) {
+        HivePrivilegeObjectType type = HivePrivilegeObjectType.TABLE_OR_VIEW;
+        HivePrivObjectActionType objectActionType = HivePrivilegeObject.HivePrivObjectActionType.OTHER;
+        HivePrivilegeObject hivePrivilegeObject =
+            new HivePrivilegeObject(type, table.getDbName(), table.getTableName(), null, null, objectActionType, null,
+                null);
+        ret.add(hivePrivilegeObject);
+      }
+    } else {
+      for (String tableName : tableNames) {
+        HivePrivilegeObjectType type = HivePrivilegeObjectType.TABLE_OR_VIEW;
+        HivePrivObjectActionType objectActionType = HivePrivilegeObject.HivePrivObjectActionType.OTHER;
+        HivePrivilegeObject hivePrivilegeObject =
+            new HivePrivilegeObject(type, dbName, tableName, null, null, objectActionType, null, null);
+        ret.add(hivePrivilegeObject);
+      }
+    }
+
+    LOG.debug("<== TableFilterContext.getOutputHObjs(): ret=" + ret);
+
+    return ret;
+  }
+
+  private List<HivePrivilegeObject> getOutputHObjs() {
+    return Collections.emptyList();
+  }
+
+  public List<Table> getTables() {
+    return tables;
+  }
+}
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/TestHiveMetaStoreAuthorizer.java b/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/TestHiveMetaStoreAuthorizer.java
index b9c0dcc..f99765f 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/TestHiveMetaStoreAuthorizer.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/metastore/TestHiveMetaStoreAuthorizer.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.ql.security.authorization.plugin.metastore;
 
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.metastore.ColumnType;
@@ -283,4 +284,50 @@ public class TestHiveMetaStoreAuthorizer {
       // no Exceptions for superuser as hive is allowed CREATE CATALOG operation
     }
   }
+
+  @Test
+  public void testNShowDatabaseAuthorizedUser() throws Exception {
+    UserGroupInformation.setLoginUser(UserGroupInformation.createRemoteUser(authorizedUser));
+    try {
+      hmsHandler.get_all_databases();
+    } catch (Exception e) {
+      // no Exceptions for show database as authorized user.
+    }
+  }
+
+  @Test
+  public void testOShowDatabaseUnauthorizedUser() throws Exception {
+    UserGroupInformation.setLoginUser(UserGroupInformation.createRemoteUser(unAuthorizedUser));
+    try {
+      hmsHandler.get_all_databases();
+    } catch (Exception e) {
+      String err = e.getMessage();
+      if (StringUtils.isNotEmpty(err)) {
+        assert(true);
+      }
+    }
+  }
+
+  @Test
+  public void testPShowTablesAuthorizedUser() throws Exception {
+    UserGroupInformation.setLoginUser(UserGroupInformation.createRemoteUser(authorizedUser));
+    try {
+      hmsHandler.get_all_tables("default");
+    } catch (Exception e) {
+      // no Exceptions for show tables as authorized user.
+    }
+  }
+
+  @Test
+  public void testQShowTablesUnauthorizedUser() throws Exception {
+    UserGroupInformation.setLoginUser(UserGroupInformation.createRemoteUser(unAuthorizedUser));
+    try {
+      hmsHandler.get_all_tables("default");
+    } catch (Exception e) {
+      String err = e.getMessage();
+      if (StringUtils.isNotEmpty(err)) {
+        assert(true);
+      }
+    }
+  }
 }